From d38df2ae92920cbbca38159cdd29c0c9bb5979f0 Mon Sep 17 00:00:00 2001 From: garmes-gdev Date: Thu, 22 Dec 2016 12:10:29 +0100 Subject: [PATCH 001/101] KAFKA-3355 GetOffsetShell command doesn't work with SASL enabled Kafka --- .../main/scala/kafka/admin/AdminClient.scala | 12 + .../scala/kafka/tools/GetOffsetShell.scala | 227 ++++++++++++++---- 2 files changed, 187 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index a7e7ebc9064e6..7088447db0f3a 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -154,6 +154,18 @@ class AdminClient(val time: Time, ConsumerGroupSummary(metadata.state, metadata.protocol, Some(consumers), coordinator) } + + def getTopicListOffset(req: ListOffsetRequest, node: Node): Map[TopicPartition, ListOffsetResponse.PartitionData] = { + val responseBody = send(node, ApiKeys.LIST_OFFSETS, req) + val response = responseBody.asInstanceOf[ListOffsetResponse] + } + + + def getMetadata(req: MetadataRequest, node: Node): MetadataResponse = { + val responseBody = send(node, ApiKeys.METADATA, req) + responseBody.asInstanceOf[MetadataResponse] + } + def close() { client.close() } diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 979354bc844f7..89927c1b3d09a 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -18,92 +18,215 @@ */ package kafka.tools +import java.util.Properties + import kafka.consumer._ import joptsimple._ -import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} +import kafka.admin.AdminClient +import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo} import kafka.common.TopicAndPartition import kafka.client.ClientUtils -import kafka.utils.{ToolsUtils, CommandLineUtils} +import kafka.cluster.BrokerEndPoint +import kafka.utils.{CommandLineUtils, ToolsUtils} +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.requests.{ListOffsetRequest, MetadataRequest} +import org.apache.kafka.common.utils.Utils + +import scala.collection.Seq +import scala.util.Random object GetOffsetShell { + val clientId = "GetOffsetShell" + def main(args: Array[String]): Unit = { val parser = new OptionParser val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.") - .withRequiredArg - .describedAs("hostname:port,...,hostname:port") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("hostname:port,...,hostname:port") + .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to get offset from.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) val partitionOpt = parser.accepts("partitions", "comma separated list of partition ids. If not specified, it will find offsets for all partitions") - .withRequiredArg - .describedAs("partition ids") - .ofType(classOf[String]) - .defaultsTo("") - val timeOpt = parser.accepts("time", "timestamp of the offsets before that") - .withRequiredArg - .describedAs("timestamp/-1(latest)/-2(earliest)") - .ofType(classOf[java.lang.Long]) - .defaultsTo(-1) + .withRequiredArg + .describedAs("partition ids") + .ofType(classOf[String]) + .defaultsTo("") + val timeOpt = parser.accepts("time", " REQUIRED: timestamp of the offsets before that") + .withRequiredArg + .describedAs("timestamp/-1(latest)/-2(earliest)") + .ofType(classOf[java.lang.Long]) val nOffsetsOpt = parser.accepts("offsets", "number of offsets returned") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1000) - - if(args.length == 0) + .withRequiredArg + .describedAs("ms") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1000) + val newApiOpt = parser.accepts("new-api", "Use new api. This is the default.") + .withRequiredArg + .describedAs("boolean") + .ofType(classOf[java.lang.Boolean]) + .defaultsTo(true) + val commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client.") + .withRequiredArg + .describedAs("command config property file") + .ofType(classOf[String]) + + if(args.length == 0) CommandLineUtils.printUsageAndDie(parser, "An interactive shell for getting consumer offsets.") val options = parser.parse(args : _*) - CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt) + CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt, timeOpt) + - val clientId = "GetOffsetShell" val brokerList = options.valueOf(brokerListOpt) ToolsUtils.validatePortOrDie(parser, brokerList) val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) val topic = options.valueOf(topicOpt) - var partitionList = options.valueOf(partitionOpt) - var time = options.valueOf(timeOpt).longValue + val partitionList = options.valueOf(partitionOpt) + val time = options.valueOf(timeOpt).longValue val nOffsets = options.valueOf(nOffsetsOpt).intValue val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() + val useNewApi = options.valueOf(newApiOpt).booleanValue() + val commandConfig = if (options.has(commandConfigOpt)) { + Utils.loadProps(options.valueOf(commandConfigOpt)) + } else new Properties() + + + if (useNewApi) { + System.err.println("Note: This will use the new Java Api.\n") + new NewGetOffsetShell(brokerList,metadataTargetBrokers,topic, partitionList, time, nOffsets ,maxWaitMs , commandConfig ) + } else { + System.err.println("Note: This will use the old Java API (use consumer to get offsets).\n") + new OldGetOffsetShell(metadataTargetBrokers,topic, partitionList, time, nOffsets ,maxWaitMs ) + } + + + } + + + + +} + + +class OldGetOffsetShell(val metadataTargetBrokers: Seq[BrokerEndPoint], + val topic: String, + var partitionList: String, + var time: java.lang.Long, + val nOffsets: Int, + val maxWaitMs: Int){ + + + val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, GetOffsetShell.clientId, maxWaitMs).topicsMetadata + if(topicsMetadata.size != 1 || !topicsMetadata(0).topic.equals(topic)) { + System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + + "kafka-list-topic.sh to verify") + System.exit(1) + } + + val partitions = + if(partitionList == "") { + topicsMetadata.head.partitionsMetadata.map(_.partitionId) + } else { + partitionList.split(",").map(_.toInt).toSeq + } + + partitions.foreach { partitionId => + val partitionMetadataOpt = topicsMetadata.head.partitionsMetadata.find(_.partitionId == partitionId) + partitionMetadataOpt match { + case Some(metadata) => + metadata.leader match { + case Some(leader) => + val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, GetOffsetShell.clientId) + val topicAndPartition = TopicAndPartition(topic, partitionId) + val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) + val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata - if(topicsMetadata.size != 1 || !topicsMetadata.head.topic.equals(topic)) { - System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + - "kafka-list-topic.sh to verify") - System.exit(1) + println("%s:%d:%s".format(topic, partitionId, offsets.mkString(","))) + case None => System.err.println("Error: partition %d does not have a leader. Skip getting offsets".format(partitionId)) + } + case None => System.err.println("Error: partition %d does not exist".format(partitionId)) } + } + + +} +class NewGetOffsetShell(val brokerList: String, + val metadataTargetBrokers: Seq[BrokerEndPoint], + val topic: String, + var partitionList: String, + var time: java.lang.Long, + val nOffsets: java.lang.Integer, + val maxWaitMs: java.lang.Integer, + val commandConfig: Properties){ + + + private def createAdminClient(props: Properties): AdminClient = { + AdminClient.create(props) + } + private def getNode(brokerEndPoint: BrokerEndPoint): Node = { + new Node(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + } + + + import scala.collection.JavaConverters._ + + commandConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList) + val adminClient = createAdminClient(commandConfig) + + val shuffledBrokers = Random.shuffle(metadataTargetBrokers) + val metadataRes = adminClient.getMetadata(new MetadataRequest(List(topic).asJava), getNode(metadataTargetBrokers(0))) + + + if(metadataRes.errors.containsKey(topic)){ + metadataRes.errors().get(topic).exception() + }else{ + + val topicsPartitions = metadataRes.cluster().availablePartitionsForTopic(topic).asScala + val partitions = if(partitionList == "") { - topicsMetadata.head.partitionsMetadata.map(_.partitionId) + topicsPartitions.map(_.partition()) } else { partitionList.split(",").map(_.toInt).toSeq } - partitions.foreach { partitionId => - val partitionMetadataOpt = topicsMetadata.head.partitionsMetadata.find(_.partitionId == partitionId) - partitionMetadataOpt match { - case Some(metadata) => - metadata.leader match { - case Some(leader) => - val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId) - val topicAndPartition = TopicAndPartition(topic, partitionId) - val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) - val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets - - println("%s:%d:%s".format(topic, partitionId, offsets.mkString(","))) - case None => System.err.println("Error: partition %d does not have a leader. Skip getting offsets".format(partitionId)) - } - case None => System.err.println("Error: partition %d does not exist".format(partitionId)) + + partitions.foreach { partitionId: Int => + val partitionMetadata = topicsPartitions.toList.find(_.partition == partitionId) + partitionMetadata match { + case Some(metadata) => { + + val partitions = Map(new TopicPartition(metadata.topic(), metadata.partition()) -> time) + + val request: ListOffsetRequest = new ListOffsetRequest(partitions.asJava, ListOffsetRequest.CONSUMER_REPLICA_ID) + + val listOffset= adminClient.getTopicListOffset(request,metadata.leader() ) + + listOffset.keys.foreach(topicPartition =>{ + val data = listOffset.get(topicPartition).get + + if (data.errorCode == Errors.NONE.code) { + println("%s:%d:%s".format(topic, partitionId, data.offset )) + } else { + val errormessage =Errors.forCode(data.errorCode ).exception.getMessage + println(s"Attempt to fetch offsets for partition $topicPartition failed due to: $errormessage") + } + }) + + } } } } + } From a36aa06b0a2a1e7838e272da29e67b12bb354a51 Mon Sep 17 00:00:00 2001 From: garmes-gdev Date: Thu, 5 Jan 2017 12:56:44 +0100 Subject: [PATCH 002/101] KAFKA-3355 GetOffsetShell command doesn't work with SASL enabled. --- .../main/scala/kafka/admin/AdminClient.scala | 2 +- .../scala/kafka/tools/GetOffsetShell.scala | 176 +++++------------- tests/kafkatest/services/kafka/kafka.py | 4 +- .../tests/core/get_offset_shell_test.py | 4 +- 4 files changed, 49 insertions(+), 137 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 7088447db0f3a..e6bc3281a5590 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -157,7 +157,7 @@ class AdminClient(val time: Time, def getTopicListOffset(req: ListOffsetRequest, node: Node): Map[TopicPartition, ListOffsetResponse.PartitionData] = { val responseBody = send(node, ApiKeys.LIST_OFFSETS, req) - val response = responseBody.asInstanceOf[ListOffsetResponse] + responseBody.asInstanceOf[ListOffsetResponse].responseData().asScala.toMap } diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 89927c1b3d09a..218d22ebf9287 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -19,12 +19,8 @@ package kafka.tools import java.util.Properties - -import kafka.consumer._ import joptsimple._ import kafka.admin.AdminClient -import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo} -import kafka.common.TopicAndPartition import kafka.client.ClientUtils import kafka.cluster.BrokerEndPoint import kafka.utils.{CommandLineUtils, ToolsUtils} @@ -33,8 +29,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.requests.{ListOffsetRequest, MetadataRequest} import org.apache.kafka.common.utils.Utils - -import scala.collection.Seq +import scala.collection.JavaConverters._ import scala.util.Random @@ -42,6 +37,14 @@ object GetOffsetShell { val clientId = "GetOffsetShell" + private def createAdminClient(props: Properties): AdminClient = { + AdminClient.create(props) + } + private def getNode(brokerEndPoint: BrokerEndPoint): Node = { + new Node(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + } + + def main(args: Array[String]): Unit = { val parser = new OptionParser val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.") @@ -61,21 +64,7 @@ object GetOffsetShell { .withRequiredArg .describedAs("timestamp/-1(latest)/-2(earliest)") .ofType(classOf[java.lang.Long]) - val nOffsetsOpt = parser.accepts("offsets", "number of offsets returned") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1000) - val newApiOpt = parser.accepts("new-api", "Use new api. This is the default.") - .withRequiredArg - .describedAs("boolean") - .ofType(classOf[java.lang.Boolean]) - .defaultsTo(true) + val commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client.") .withRequiredArg .describedAs("command config property file") @@ -95,138 +84,61 @@ object GetOffsetShell { val topic = options.valueOf(topicOpt) val partitionList = options.valueOf(partitionOpt) val time = options.valueOf(timeOpt).longValue - val nOffsets = options.valueOf(nOffsetsOpt).intValue - val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() - val useNewApi = options.valueOf(newApiOpt).booleanValue() val commandConfig = if (options.has(commandConfigOpt)) { Utils.loadProps(options.valueOf(commandConfigOpt)) } else new Properties() - if (useNewApi) { - System.err.println("Note: This will use the new Java Api.\n") - new NewGetOffsetShell(brokerList,metadataTargetBrokers,topic, partitionList, time, nOffsets ,maxWaitMs , commandConfig ) - } else { - System.err.println("Note: This will use the old Java API (use consumer to get offsets).\n") - new OldGetOffsetShell(metadataTargetBrokers,topic, partitionList, time, nOffsets ,maxWaitMs ) - } + commandConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList) + val adminClient = createAdminClient(commandConfig) + val shuffledBrokers = Random.shuffle(metadataTargetBrokers) + val metadataRes = adminClient.getMetadata(new MetadataRequest(List(topic).asJava), getNode(shuffledBrokers(0))) - } + if(metadataRes.errors.containsKey(topic)){ + metadataRes.errors().get(topic).exception() + }else{ + val topicsPartitions = metadataRes.cluster().availablePartitionsForTopic(topic).asScala + val partitions = + if(partitionList == "") { + topicsPartitions.map(_.partition()) + } else { + partitionList.split(",").map(_.toInt).toSeq + } -} + partitions.foreach { partitionId: Int => + val partitionMetadata = topicsPartitions.toList.find(_.partition == partitionId) + partitionMetadata match { + case Some(metadata) => { + val partitions:java.util.Map[TopicPartition, java.lang.Long] = Map(new TopicPartition(metadata.topic(), metadata.partition()) -> + java.lang.Long.valueOf(time)).asJava -class OldGetOffsetShell(val metadataTargetBrokers: Seq[BrokerEndPoint], - val topic: String, - var partitionList: String, - var time: java.lang.Long, - val nOffsets: Int, - val maxWaitMs: Int){ + val request: ListOffsetRequest = new ListOffsetRequest(partitions, ListOffsetRequest.CONSUMER_REPLICA_ID) + val listOffset= adminClient.getTopicListOffset(request,metadata.leader() ) - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, GetOffsetShell.clientId, maxWaitMs).topicsMetadata - if(topicsMetadata.size != 1 || !topicsMetadata(0).topic.equals(topic)) { - System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + - "kafka-list-topic.sh to verify") - System.exit(1) - } + listOffset.keys.foreach(topicPartition =>{ + val data = listOffset.get(topicPartition).get - val partitions = - if(partitionList == "") { - topicsMetadata.head.partitionsMetadata.map(_.partitionId) - } else { - partitionList.split(",").map(_.toInt).toSeq - } + if (data.errorCode == Errors.NONE.code) { + println("%s:%d:%s".format(topic, partitionId, data.offset )) + } else { + val errormessage =Errors.forCode(data.errorCode ).exception.getMessage + println(s"Attempt to fetch offsets for partition $topicPartition failed due to: $errormessage") + } + }) - partitions.foreach { partitionId => - val partitionMetadataOpt = topicsMetadata.head.partitionsMetadata.find(_.partitionId == partitionId) - partitionMetadataOpt match { - case Some(metadata) => - metadata.leader match { - case Some(leader) => - val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, GetOffsetShell.clientId) - val topicAndPartition = TopicAndPartition(topic, partitionId) - val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) - val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets - - println("%s:%d:%s".format(topic, partitionId, offsets.mkString(","))) - case None => System.err.println("Error: partition %d does not have a leader. Skip getting offsets".format(partitionId)) + } } - case None => System.err.println("Error: partition %d does not exist".format(partitionId)) + } } - } - - -} -class NewGetOffsetShell(val brokerList: String, - val metadataTargetBrokers: Seq[BrokerEndPoint], - val topic: String, - var partitionList: String, - var time: java.lang.Long, - val nOffsets: java.lang.Integer, - val maxWaitMs: java.lang.Integer, - val commandConfig: Properties){ - - private def createAdminClient(props: Properties): AdminClient = { - AdminClient.create(props) } - private def getNode(brokerEndPoint: BrokerEndPoint): Node = { - new Node(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) - } - - - import scala.collection.JavaConverters._ - - commandConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList) - val adminClient = createAdminClient(commandConfig) - - val shuffledBrokers = Random.shuffle(metadataTargetBrokers) - val metadataRes = adminClient.getMetadata(new MetadataRequest(List(topic).asJava), getNode(metadataTargetBrokers(0))) - - - if(metadataRes.errors.containsKey(topic)){ - metadataRes.errors().get(topic).exception() - }else{ - - val topicsPartitions = metadataRes.cluster().availablePartitionsForTopic(topic).asScala - - val partitions = - if(partitionList == "") { - topicsPartitions.map(_.partition()) - } else { - partitionList.split(",").map(_.toInt).toSeq - } - - partitions.foreach { partitionId: Int => - val partitionMetadata = topicsPartitions.toList.find(_.partition == partitionId) - partitionMetadata match { - case Some(metadata) => { - - val partitions = Map(new TopicPartition(metadata.topic(), metadata.partition()) -> time) - - val request: ListOffsetRequest = new ListOffsetRequest(partitions.asJava, ListOffsetRequest.CONSUMER_REPLICA_ID) - - val listOffset= adminClient.getTopicListOffset(request,metadata.leader() ) - - listOffset.keys.foreach(topicPartition =>{ - val data = listOffset.get(topicPartition).get +} - if (data.errorCode == Errors.NONE.code) { - println("%s:%d:%s".format(topic, partitionId, data.offset )) - } else { - val errormessage =Errors.forCode(data.errorCode ).exception.getMessage - println(s"Attempt to fetch offsets for partition $topicPartition failed due to: $errormessage") - } - }) - } - } - } - } -} diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index f773d8d6843d8..76c058e2410b8 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -562,12 +562,12 @@ def is_registered(self, node): self.logger.debug("Broker info: %s", broker_info) return broker_info is not None - def get_offset_shell(self, topic, partitions, max_wait_ms, offsets, time): + def get_offset_shell(self, topic, partitions, time): node = self.nodes[0] cmd = self.path.script("kafka-run-class.sh", node) cmd += " kafka.tools.GetOffsetShell" - cmd += " --topic %s --broker-list %s --max-wait-ms %s --offsets %s --time %s" % (topic, self.bootstrap_servers(self.security_protocol), max_wait_ms, offsets, time) + cmd += " --topic %s --broker-list %s --time %s" % (topic, self.bootstrap_servers(self.security_protocol), time) if partitions: cmd += ' --partitions %s' % partitions diff --git a/tests/kafkatest/tests/core/get_offset_shell_test.py b/tests/kafkatest/tests/core/get_offset_shell_test.py index e45365d32cf31..d139f766e2c13 100644 --- a/tests/kafkatest/tests/core/get_offset_shell_test.py +++ b/tests/kafkatest/tests/core/get_offset_shell_test.py @@ -80,7 +80,7 @@ def test_get_offset_shell(self, security_protocol='PLAINTEXT'): self.start_producer() # Assert that offset fetched without any consumers consuming is 0 - assert self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, 0) + assert self.kafka.get_offset_shell(TOPIC, None, -1), "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, 0) self.start_consumer(security_protocol) @@ -89,5 +89,5 @@ def test_get_offset_shell(self, security_protocol='PLAINTEXT'): wait_until(lambda: self.consumer.alive(node), timeout_sec=10, backoff_sec=.2, err_msg="Consumer was too slow to start") # Assert that offset is correctly indicated by GetOffsetShell tool - wait_until(lambda: "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, MAX_MESSAGES) in self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), timeout_sec=10, + wait_until(lambda: "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, MAX_MESSAGES) in self.kafka.get_offset_shell(TOPIC, None, -1), timeout_sec=10, err_msg="Timed out waiting to reach expected offset.") \ No newline at end of file From 6c839395b7b43d8e9c0dfbb3c12470fc9284a94a Mon Sep 17 00:00:00 2001 From: Aegeaner Date: Thu, 16 Feb 2017 13:44:08 -0800 Subject: [PATCH 003/101] =?UTF-8?q?KAFKA-4709=EF=BC=9AError=20message=20fr?= =?UTF-8?q?om=20Struct.validate()=20should=20include=20the=20name=20of=20t?= =?UTF-8?q?he=20offending=20field.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit https://issues.apache.org/jira/browse/KAFKA-4709 Author: Aegeaner Reviewers: Dong Lin, Guozhang Wang Closes #2521 from Aegeaner/KAFKA-4709 --- .../kafka/connect/data/ConnectSchema.java | 15 +++- .../org/apache/kafka/connect/data/Struct.java | 2 +- .../apache/kafka/connect/data/FakeSchema.java | 83 +++++++++++++++++++ .../apache/kafka/connect/data/StructTest.java | 33 ++++++++ 4 files changed, 129 insertions(+), 4 deletions(-) create mode 100644 connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index d1fd9cdaa7aa6..e052534a7a955 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -207,9 +207,14 @@ public Schema valueSchema() { * @param value value to test */ public static void validateValue(Schema schema, Object value) { + validateValue(null, schema, value); + } + + public static void validateValue(String name, Schema schema, Object value) { if (value == null) { if (!schema.isOptional()) - throw new DataException("Invalid value: null used for required field"); + throw new DataException("Invalid value: null used for required field: \"" + name + + "\", schema type: " + schema.type()); else return; } @@ -220,7 +225,9 @@ public static void validateValue(Schema schema, Object value) { expectedClasses = SCHEMA_TYPE_CLASSES.get(schema.type()); if (expectedClasses == null) - throw new DataException("Invalid Java object for schema type " + schema.type() + ": " + value.getClass()); + throw new DataException("Invalid Java object for schema type " + schema.type() + + ": " + value.getClass() + + " for field: \"" + name + "\""); boolean foundMatch = false; for (Class expectedClass : expectedClasses) { @@ -230,7 +237,9 @@ public static void validateValue(Schema schema, Object value) { } } if (!foundMatch) - throw new DataException("Invalid Java object for schema type " + schema.type() + ": " + value.getClass()); + throw new DataException("Invalid Java object for schema type " + schema.type() + + ": " + value.getClass() + + " for field: \"" + name + "\""); switch (schema.type()) { case STRUCT: diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java index 698c6ee8cb895..200a1c00a9df5 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java @@ -229,7 +229,7 @@ public void validate() { Object value = values[field.index()]; if (value == null && (fieldSchema.isOptional() || fieldSchema.defaultValue() != null)) continue; - ConnectSchema.validateValue(fieldSchema, value); + ConnectSchema.validateValue(field.name(), fieldSchema, value); } } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java b/connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java new file mode 100644 index 0000000000000..ff2e24f6a2b01 --- /dev/null +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.data; + +import java.util.List; +import java.util.Map; + +public class FakeSchema implements Schema { + @Override + public Type type() { + return null; + } + + @Override + public boolean isOptional() { + return false; + } + + @Override + public Object defaultValue() { + return null; + } + + @Override + public String name() { + return "fake"; + } + + @Override + public Integer version() { + return null; + } + + @Override + public String doc() { + return null; + } + + @Override + public Map parameters() { + return null; + } + + @Override + public Schema keySchema() { + return null; + } + + @Override + public Schema valueSchema() { + return null; + } + + @Override + public List fields() { + return null; + } + + @Override + public Field field(String fieldName) { + return null; + } + + @Override + public Schema schema() { + return null; + } +} diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java index 11c9fb08fc3b0..82f6d89a90f68 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java @@ -18,7 +18,9 @@ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -234,4 +236,35 @@ public void testEquals() { assertEquals(struct1, struct2); assertNotEquals(struct1, struct3); } + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testValidateStructWithNullValue() { + Schema schema = SchemaBuilder.struct() + .field("one", Schema.STRING_SCHEMA) + .field("two", Schema.STRING_SCHEMA) + .field("three", Schema.STRING_SCHEMA) + .build(); + + Struct struct = new Struct(schema); + thrown.expect(DataException.class); + thrown.expectMessage("Invalid value: null used for required field: \"one\", schema type: STRING"); + struct.validate(); + } + + @Test + public void testValidateFieldWithInvalidValueType() { + String fieldName = "field"; + FakeSchema fakeSchema = new FakeSchema(); + + thrown.expect(DataException.class); + thrown.expectMessage("Invalid Java object for schema type null: class java.lang.Object for field: \"field\""); + ConnectSchema.validateValue(fieldName, fakeSchema, new Object()); + + thrown.expect(DataException.class); + thrown.expectMessage("Invalid Java object for schema type INT8: class java.lang.Object for field: \"field\""); + ConnectSchema.validateValue(fieldName, Schema.INT8_SCHEMA, new Object()); + } } From 022d2017a781286d223d80d8eae505fbc3fb369c Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 16 Feb 2017 15:22:55 -0800 Subject: [PATCH 004/101] MINOR: Move compression stream construction into CompressionType Author: Jason Gustafson Reviewers: Ismael Juma Closes #2536 from hachikuji/minor-move-compression-io-construction --- .../common/record/ByteBufferInputStream.java | 36 ++--- .../common/record/ByteBufferOutputStream.java | 53 +++---- .../kafka/common/record/CompressionType.java | 147 +++++++++++++++++- .../common/record/MemoryRecordsBuilder.java | 137 +--------------- .../apache/kafka/common/record/Record.java | 2 +- .../kafka/common/record/RecordsIterator.java | 3 +- .../consumer/internals/FetcherTest.java | 3 +- 7 files changed, 191 insertions(+), 190 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java index 37e4766aafce7..fca45cf7f4b61 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java @@ -16,41 +16,33 @@ */ package org.apache.kafka.common.record; -import java.io.DataInputStream; import java.io.InputStream; import java.nio.ByteBuffer; /** * A byte buffer backed input inputStream */ -public final class ByteBufferInputStream extends DataInputStream { +public final class ByteBufferInputStream extends InputStream { + private final ByteBuffer buffer; public ByteBufferInputStream(ByteBuffer buffer) { - super(new UnderlyingInputStream(buffer)); + this.buffer = buffer; } - private static final class UnderlyingInputStream extends InputStream { - private final ByteBuffer buffer; - - public UnderlyingInputStream(ByteBuffer buffer) { - this.buffer = buffer; + public int read() { + if (!buffer.hasRemaining()) { + return -1; } + return buffer.get() & 0xFF; + } - public int read() { - if (!buffer.hasRemaining()) { - return -1; - } - return buffer.get() & 0xFF; + public int read(byte[] bytes, int off, int len) { + if (!buffer.hasRemaining()) { + return -1; } - public int read(byte[] bytes, int off, int len) { - if (!buffer.hasRemaining()) { - return -1; - } - - len = Math.min(len, buffer.remaining()); - buffer.get(bytes, off, len); - return len; - } + len = Math.min(len, buffer.remaining()); + buffer.get(bytes, off, len); + return len; } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java index 3fb7f49f5d643..13609d8baa1f0 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java @@ -16,54 +16,43 @@ */ package org.apache.kafka.common.record; -import java.io.DataOutputStream; import java.io.OutputStream; import java.nio.ByteBuffer; /** * A byte buffer backed output outputStream */ -public class ByteBufferOutputStream extends DataOutputStream { +public class ByteBufferOutputStream extends OutputStream { private static final float REALLOCATION_FACTOR = 1.1f; + private ByteBuffer buffer; + public ByteBufferOutputStream(ByteBuffer buffer) { - super(new UnderlyingOutputStream(buffer)); + this.buffer = buffer; } - public ByteBuffer buffer() { - return ((UnderlyingOutputStream) out).buffer; + public void write(int b) { + if (buffer.remaining() < 1) + expandBuffer(buffer.capacity() + 1); + buffer.put((byte) b); } - public static class UnderlyingOutputStream extends OutputStream { - private ByteBuffer buffer; - - public UnderlyingOutputStream(ByteBuffer buffer) { - this.buffer = buffer; - } - - public void write(int b) { - if (buffer.remaining() < 1) - expandBuffer(buffer.capacity() + 1); - buffer.put((byte) b); - } - - public void write(byte[] bytes, int off, int len) { - if (buffer.remaining() < len) - expandBuffer(buffer.capacity() + len); - buffer.put(bytes, off, len); - } + public void write(byte[] bytes, int off, int len) { + if (buffer.remaining() < len) + expandBuffer(buffer.capacity() + len); + buffer.put(bytes, off, len); + } - public ByteBuffer buffer() { - return buffer; - } + public ByteBuffer buffer() { + return buffer; + } - private void expandBuffer(int size) { - int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size); - ByteBuffer temp = ByteBuffer.allocate(expandSize); - temp.put(buffer.array(), buffer.arrayOffset(), buffer.position()); - buffer = temp; - } + private void expandBuffer(int size) { + int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size); + ByteBuffer temp = ByteBuffer.allocate(expandSize); + temp.put(buffer.array(), buffer.arrayOffset(), buffer.position()); + buffer = temp; } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java index e1d4754ed0411..62265dc77d7fa 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java @@ -16,11 +16,91 @@ */ package org.apache.kafka.common.record; +import org.apache.kafka.common.KafkaException; + +import java.io.InputStream; +import java.io.OutputStream; +import java.lang.reflect.Constructor; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; + /** * The compression type to use */ public enum CompressionType { - NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f); + NONE(0, "none", 1.0f) { + @Override + public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion, int bufferSize) { + return buffer; + } + + @Override + public InputStream wrapForInput(ByteBufferInputStream buffer, byte messageVersion) { + return buffer; + } + }, + + GZIP(1, "gzip", 0.5f) { + @Override + public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion, int bufferSize) { + try { + return new GZIPOutputStream(buffer, bufferSize); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + @Override + public InputStream wrapForInput(ByteBufferInputStream buffer, byte messageVersion) { + try { + return new GZIPInputStream(buffer); + } catch (Exception e) { + throw new KafkaException(e); + } + } + }, + + SNAPPY(2, "snappy", 0.5f) { + @Override + public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion, int bufferSize) { + try { + return (OutputStream) SNAPPY_OUTPUT_STREAM_SUPPLIER.get().newInstance(buffer, bufferSize); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + @Override + public InputStream wrapForInput(ByteBufferInputStream buffer, byte messageVersion) { + try { + return (InputStream) SNAPPY_INPUT_STREAM_SUPPLIER.get().newInstance(buffer); + } catch (Exception e) { + throw new KafkaException(e); + } + } + }, + + LZ4(3, "lz4", 0.5f) { + @Override + public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion, int bufferSize) { + try { + return (OutputStream) LZ4_OUTPUT_STREAM_SUPPLIER.get().newInstance(buffer, + messageVersion == Record.MAGIC_VALUE_V0); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + @Override + public InputStream wrapForInput(ByteBufferInputStream buffer, byte messageVersion) { + try { + return (InputStream) LZ4_INPUT_STREAM_SUPPLIER.get().newInstance(buffer, + messageVersion == Record.MAGIC_VALUE_V0); + } catch (Exception e) { + throw new KafkaException(e); + } + } + }; public final int id; public final String name; @@ -32,6 +112,10 @@ public enum CompressionType { this.rate = rate; } + public abstract OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion, int bufferSize); + + public abstract InputStream wrapForInput(ByteBufferInputStream buffer, byte messageVersion); + public static CompressionType forId(int id) { switch (id) { case 0: @@ -60,4 +144,65 @@ else if (LZ4.name.equals(name)) throw new IllegalArgumentException("Unknown compression name: " + name); } + // dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression + // caching constructors to avoid invoking of Class.forName method for each batch + private static final MemoizingConstructorSupplier SNAPPY_OUTPUT_STREAM_SUPPLIER = new MemoizingConstructorSupplier(new ConstructorSupplier() { + @Override + public Constructor get() throws ClassNotFoundException, NoSuchMethodException { + return Class.forName("org.xerial.snappy.SnappyOutputStream") + .getConstructor(OutputStream.class, Integer.TYPE); + } + }); + + private static final MemoizingConstructorSupplier LZ4_OUTPUT_STREAM_SUPPLIER = new MemoizingConstructorSupplier(new ConstructorSupplier() { + @Override + public Constructor get() throws ClassNotFoundException, NoSuchMethodException { + return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream") + .getConstructor(OutputStream.class, Boolean.TYPE); + } + }); + + private static final MemoizingConstructorSupplier SNAPPY_INPUT_STREAM_SUPPLIER = new MemoizingConstructorSupplier(new ConstructorSupplier() { + @Override + public Constructor get() throws ClassNotFoundException, NoSuchMethodException { + return Class.forName("org.xerial.snappy.SnappyInputStream") + .getConstructor(InputStream.class); + } + }); + + private static final MemoizingConstructorSupplier LZ4_INPUT_STREAM_SUPPLIER = new MemoizingConstructorSupplier(new ConstructorSupplier() { + @Override + public Constructor get() throws ClassNotFoundException, NoSuchMethodException { + return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream") + .getConstructor(InputStream.class, Boolean.TYPE); + } + }); + + private interface ConstructorSupplier { + Constructor get() throws ClassNotFoundException, NoSuchMethodException; + } + + // this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier} + private static class MemoizingConstructorSupplier { + final ConstructorSupplier delegate; + transient volatile boolean initialized; + transient Constructor value; + + public MemoizingConstructorSupplier(ConstructorSupplier delegate) { + this.delegate = delegate; + } + + public Constructor get() throws NoSuchMethodException, ClassNotFoundException { + if (!initialized) { + synchronized (this) { + if (!initialized) { + value = delegate.get(); + initialized = true; + } + } + } + return value; + } + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index a17c0e9d00d99..39f21a99de496 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -18,15 +18,9 @@ import org.apache.kafka.common.KafkaException; -import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.lang.reflect.Constructor; import java.nio.ByteBuffer; -import java.util.zip.GZIPInputStream; -import java.util.zip.GZIPOutputStream; /** * This class is used to write new log data in memory, i.e. this is the write path for {@link MemoryRecords}. @@ -34,10 +28,9 @@ * format conversion. */ public class MemoryRecordsBuilder { - - static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f; - static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f; - static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024; + private static final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f; + private static final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f; + private static final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024; private static final float[] TYPE_TO_RATE; @@ -51,40 +44,6 @@ public class MemoryRecordsBuilder { } } - // dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression - // caching constructors to avoid invoking of Class.forName method for each batch - private static MemoizingConstructorSupplier snappyOutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() { - @Override - public Constructor get() throws ClassNotFoundException, NoSuchMethodException { - return Class.forName("org.xerial.snappy.SnappyOutputStream") - .getConstructor(OutputStream.class, Integer.TYPE); - } - }); - - private static MemoizingConstructorSupplier lz4OutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() { - @Override - public Constructor get() throws ClassNotFoundException, NoSuchMethodException { - return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream") - .getConstructor(OutputStream.class, Boolean.TYPE); - } - }); - - private static MemoizingConstructorSupplier snappyInputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() { - @Override - public Constructor get() throws ClassNotFoundException, NoSuchMethodException { - return Class.forName("org.xerial.snappy.SnappyInputStream") - .getConstructor(InputStream.class); - } - }); - - private static MemoizingConstructorSupplier lz4InputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() { - @Override - public Constructor get() throws ClassNotFoundException, NoSuchMethodException { - return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream") - .getConstructor(InputStream.class, Boolean.TYPE); - } - }); - private final TimestampType timestampType; private final CompressionType compressionType; private final DataOutputStream appendStream; @@ -146,7 +105,8 @@ public MemoryRecordsBuilder(ByteBuffer buffer, // create the stream bufferStream = new ByteBufferOutputStream(buffer); - appendStream = wrapForOutput(bufferStream, compressionType, magic, COMPRESSION_DEFAULT_BUFFER_SIZE); + appendStream = new DataOutputStream(compressionType.wrapForOutput(bufferStream, magic, + COMPRESSION_DEFAULT_BUFFER_SIZE)); } public ByteBuffer buffer() { @@ -399,93 +359,6 @@ public int sizeInBytes() { return builtRecords != null ? builtRecords.sizeInBytes() : estimatedBytesWritten(); } - private static DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, byte messageVersion, int bufferSize) { - try { - switch (type) { - case NONE: - return buffer; - case GZIP: - return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize)); - case SNAPPY: - try { - OutputStream stream = (OutputStream) snappyOutputStreamSupplier.get().newInstance(buffer, bufferSize); - return new DataOutputStream(stream); - } catch (Exception e) { - throw new KafkaException(e); - } - case LZ4: - try { - OutputStream stream = (OutputStream) lz4OutputStreamSupplier.get().newInstance(buffer, - messageVersion == Record.MAGIC_VALUE_V0); - return new DataOutputStream(stream); - } catch (Exception e) { - throw new KafkaException(e); - } - default: - throw new IllegalArgumentException("Unknown compression type: " + type); - } - } catch (IOException e) { - throw new KafkaException(e); - } - } - - public static DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type, byte messageVersion) { - try { - switch (type) { - case NONE: - return buffer; - case GZIP: - return new DataInputStream(new GZIPInputStream(buffer)); - case SNAPPY: - try { - InputStream stream = (InputStream) snappyInputStreamSupplier.get().newInstance(buffer); - return new DataInputStream(stream); - } catch (Exception e) { - throw new KafkaException(e); - } - case LZ4: - try { - InputStream stream = (InputStream) lz4InputStreamSupplier.get().newInstance(buffer, - messageVersion == Record.MAGIC_VALUE_V0); - return new DataInputStream(stream); - } catch (Exception e) { - throw new KafkaException(e); - } - default: - throw new IllegalArgumentException("Unknown compression type: " + type); - } - } catch (IOException e) { - throw new KafkaException(e); - } - } - - private interface ConstructorSupplier { - Constructor get() throws ClassNotFoundException, NoSuchMethodException; - } - - // this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier} - private static class MemoizingConstructorSupplier { - final ConstructorSupplier delegate; - transient volatile boolean initialized; - transient Constructor value; - - public MemoizingConstructorSupplier(ConstructorSupplier delegate) { - this.delegate = delegate; - } - - public Constructor get() throws NoSuchMethodException, ClassNotFoundException { - if (!initialized) { - synchronized (this) { - if (!initialized) { - value = delegate.get(); - initialized = true; - } - } - } - return value; - } - } - public static class RecordsInfo { public final long maxTimestamp; public final long shallowOffsetOfMaxTimestamp; diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java index 9dca5448a9811..51bbe35aa9411 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Record.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java @@ -479,7 +479,7 @@ private static void write(ByteBuffer buffer, CompressionType compressionType, TimestampType timestampType) { try { - ByteBufferOutputStream out = new ByteBufferOutputStream(buffer); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buffer)); write(out, magic, timestamp, key, value, compressionType, timestampType); } catch (IOException e) { throw new KafkaException(e); diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java index 792a8575f44f2..07c919750c93e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java @@ -147,7 +147,8 @@ public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, i CompressionType compressionType = wrapperRecord.compressionType(); ByteBuffer buffer = wrapperRecord.value(); - DataInputStream stream = MemoryRecordsBuilder.wrapForInput(new ByteBufferInputStream(buffer), compressionType, wrapperRecord.magic()); + DataInputStream stream = new DataInputStream(compressionType.wrapForInput(new ByteBufferInputStream(buffer), + wrapperRecord.magic())); LogInputStream logStream = new DataLogInputStream(stream, maxMessageSize); long wrapperRecordOffset = wrapperEntry.offset(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index b27802dc2d9a5..fec92519efc56 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -62,6 +62,7 @@ import org.junit.Before; import org.junit.Test; +import java.io.DataOutputStream; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -217,7 +218,7 @@ public byte[] deserialize(String topic, byte[] data) { @Test public void testParseInvalidRecord() throws Exception { ByteBuffer buffer = ByteBuffer.allocate(1024); - ByteBufferOutputStream out = new ByteBufferOutputStream(buffer); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buffer)); byte magic = Record.CURRENT_MAGIC_VALUE; byte[] key = "foo".getBytes(); From 505af6bfdafa4719e4faeefab4d334cb0eebf784 Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Thu, 16 Feb 2017 15:35:32 -0800 Subject: [PATCH 005/101] KAFKA-4484: Set more conservative default values on RocksDB for memory usage Lowered the default RocksDB settings for the block cache and write buffers Author: Damian Guy Reviewers: Eno Thereska, Guozhang Wang Closes #2525 from dguy/kafka-4484 --- .../apache/kafka/streams/state/internals/RocksDBStore.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 108c77252d3e3..13d96d3c9a178 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -70,8 +70,8 @@ public class RocksDBStore implements KeyValueStore { // TODO: these values should be configurable private static final CompressionType COMPRESSION_TYPE = CompressionType.NO_COMPRESSION; private static final CompactionStyle COMPACTION_STYLE = CompactionStyle.UNIVERSAL; - private static final long WRITE_BUFFER_SIZE = 32 * 1024 * 1024L; - private static final long BLOCK_CACHE_SIZE = 100 * 1024 * 1024L; + private static final long WRITE_BUFFER_SIZE = 16 * 1024 * 1024L; + private static final long BLOCK_CACHE_SIZE = 50 * 1024 * 1024L; private static final long BLOCK_SIZE = 4096L; private static final int TTL_SECONDS = TTL_NOT_USED; private static final int MAX_WRITE_BUFFERS = 3; From a88737929e880d5c6892a306d3c9b1358853b76b Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Fri, 17 Feb 2017 01:46:14 -0800 Subject: [PATCH 006/101] KAFKA-4775; Fix findbugs warnings in kafka-tools Author: Colin P. Mccabe Reviewers: Ismael Juma Closes #2559 from cmccabe/KAFKA-4775 --- .../org/apache/kafka/tools/ClientCompatibilityTest.java | 6 +++--- .../java/org/apache/kafka/tools/ThroughputThrottler.java | 6 ++---- .../java/org/apache/kafka/tools/VerifiableProducer.java | 2 +- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java index 0430c2ee7c497..663fa4a66be82 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java +++ b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java @@ -287,15 +287,15 @@ public void run() { consumer.assign(topicPartitions); consumer.seekToBeginning(topicPartitions); final Iterator iter = new Iterator() { - private final int timeoutMs = 10000; + private static final int TIMEOUT_MS = 10000; private Iterator> recordIter = null; private byte[] next = null; private byte[] fetchNext() { while (true) { long curTime = Time.SYSTEM.milliseconds(); - if (curTime - prodTimeMs > timeoutMs) - throw new RuntimeException("Timed out after " + timeoutMs + " ms."); + if (curTime - prodTimeMs > TIMEOUT_MS) + throw new RuntimeException("Timed out after " + TIMEOUT_MS + " ms."); if (recordIter == null) { ConsumerRecords records = consumer.poll(100); recordIter = records.iterator(); diff --git a/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java b/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java index b55048464572d..37ced2d56a639 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java +++ b/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java @@ -103,16 +103,14 @@ public void throttle() { // If enough sleep deficit has accumulated, sleep a little if (sleepDeficitNs >= MIN_SLEEP_NS) { long sleepStartNs = System.nanoTime(); - long currentTimeNs = sleepStartNs; try { synchronized (this) { - long elapsed = currentTimeNs - sleepStartNs; - long remaining = sleepDeficitNs - elapsed; + long remaining = sleepDeficitNs; while (!wakeup && remaining > 0) { long sleepMs = remaining / 1000000; long sleepNs = remaining - sleepMs * 1000000; this.wait(sleepMs, (int) sleepNs); - elapsed = System.nanoTime() - sleepStartNs; + long elapsed = System.nanoTime() - sleepStartNs; remaining = sleepDeficitNs - elapsed; } wakeup = false; diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java index 76176892d402e..367d9c2b645b4 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java @@ -235,7 +235,7 @@ public void send(String key, String value) { /** Returns a string to publish: ether 'valuePrefix'.'val' or 'val' **/ public String getValue(long val) { if (this.valuePrefix != null) { - return String.format("%d.%d", this.valuePrefix.intValue(), val); + return String.format("%d.%d", this.valuePrefix, val); } return String.format("%d", val); } From 1f2ee5f0a9dfc33e34362f7cd4ac45edaabe421b Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 17 Feb 2017 02:19:33 -0800 Subject: [PATCH 007/101] KAFKA-4340; Follow-up fixing system test failures and handling non default log.retention.ms Author: Jiangjie Qin Reviewers: Jun Rao , Ismael Juma Closes #2544 from becketqin/KAFKA-4340_follow_up --- .../main/scala/kafka/log/LogValidator.scala | 6 ++- .../scala/kafka/server/ConfigHandler.scala | 13 +++---- .../main/scala/kafka/server/KafkaConfig.scala | 15 +++++-- .../main/scala/kafka/server/KafkaServer.scala | 2 +- .../test/scala/unit/kafka/log/LogTest.scala | 28 ++++++++----- .../unit/kafka/server/KafkaConfigTest.scala | 2 + docs/upgrade.html | 39 ++++++++++++++++++- 7 files changed, 82 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala index 45e364cde9051..26d6e8c75ee76 100644 --- a/core/src/main/scala/kafka/log/LogValidator.scala +++ b/core/src/main/scala/kafka/log/LogValidator.scala @@ -235,9 +235,11 @@ private[kafka] object LogValidator { now: Long, timestampType: TimestampType, timestampDiffMaxMs: Long) { - if (timestampType == TimestampType.CREATE_TIME && math.abs(record.timestamp - now) > timestampDiffMaxMs) + if (timestampType == TimestampType.CREATE_TIME + && record.timestamp != Record.NO_TIMESTAMP + && math.abs(record.timestamp - now) > timestampDiffMaxMs) throw new InvalidTimestampException(s"Timestamp ${record.timestamp} of message is out of range. " + - s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}") + s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}]") if (record.timestampType == TimestampType.LOG_APPEND_TIME) throw new InvalidTimestampException(s"Invalid timestamp type in message $record. Producer should not set " + s"timestamp type to LogAppendTime.") diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index e8f13d77ed72d..8d6de8c7b2563 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -65,7 +65,7 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC && logConfig.retentionMs < logConfig.messageTimestampDifferenceMaxMs) warn(s"${LogConfig.RetentionMsProp} for topic $topic is set to ${logConfig.retentionMs}. It is smaller than " + s"${LogConfig.MessageTimestampDifferenceMaxMsProp}'s value ${logConfig.messageTimestampDifferenceMaxMs}. " + - s"This may result in potential frequent log rolling.") + s"This may result in frequent log rolling.") logs.foreach(_.config = logConfig) } @@ -98,16 +98,15 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC } def excludedConfigs(topic: String, topicConfig: Properties): Set[String] = { - val excludeConfigs: mutable.Set[String] = new mutable.HashSet[String] // Verify message format version - Option(topicConfig.getProperty(LogConfig.MessageFormatVersionProp)).foreach { versionString => + Option(topicConfig.getProperty(LogConfig.MessageFormatVersionProp)).flatMap { versionString => if (kafkaConfig.interBrokerProtocolVersion < ApiVersion(versionString)) { warn(s"Log configuration ${LogConfig.MessageFormatVersionProp} is ignored for `$topic` because `$versionString` " + s"is not compatible with Kafka inter-broker protocol version `${kafkaConfig.interBrokerProtocolVersionString}`") - excludeConfigs += LogConfig.MessageFormatVersionProp - } - } - excludeConfigs.toSet + Some(LogConfig.MessageFormatVersionProp) + } else + None + }.toSet } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 0180a2c853539..a15f034280b14 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -17,6 +17,7 @@ package kafka.server +import java.util import java.util.Properties import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1} @@ -479,7 +480,8 @@ object KafkaConfig { val LogMessageTimestampDifferenceMaxMsDoc = "The maximum difference allowed between the timestamp when a broker receives " + "a message and the timestamp specified in the message. If log.message.timestamp.type=CreateTime, a message will be rejected " + "if the difference in timestamp exceeds this threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." + - "The maximum timestamp difference allowed should be no greater than log.retention.ms to avoid unnecessarily frequent log rolling." + "The maximum timestamp difference allowed should be no greater than log.retention.ms to avoid unnecessarily frequent log rolling. For " + + "this reason, the default is the value of log.retention.ms." val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown" val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server" val MinInSyncReplicasDoc = "When a producer sets acks to \"all\" (or \"-1\"), " + @@ -690,7 +692,7 @@ object KafkaConfig { .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, MEDIUM, LogMessageFormatVersionDoc) .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) - .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) + .define(LogMessageTimestampDifferenceMaxMsProp, LONG, null, MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) .define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc) /** ********* Replication configuration ***********/ @@ -888,7 +890,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra val logMessageFormatVersionString = getString(KafkaConfig.LogMessageFormatVersionProp) val logMessageFormatVersion = ApiVersion(logMessageFormatVersionString) val logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp)) - val logMessageTimestampDifferenceMaxMs = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp) + val logMessageTimestampDifferenceMaxMs = getMessageTimestampDifferenceMaxMs /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) @@ -998,6 +1000,13 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra millis } + private def getMessageTimestampDifferenceMaxMs: Long = { + Option(getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)) match { + case Some(value) => value + case None => getLogRetentionTimeMillis + } + } + private def getMap(propName: String, propValue: String): Map[String, String] = { try { CoreUtils.parseCsvMap(propValue) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 2a247ecc4cfa3..f5d8fc6a30448 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -76,7 +76,7 @@ object KafkaServer { logProps.put(LogConfig.PreAllocateEnableProp, kafkaConfig.logPreAllocateEnable) logProps.put(LogConfig.MessageFormatVersionProp, kafkaConfig.logMessageFormatVersion.version) logProps.put(LogConfig.MessageTimestampTypeProp, kafkaConfig.logMessageTimestampType.name) - logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.logMessageTimestampDifferenceMaxMs) + logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.logMessageTimestampDifferenceMaxMs: java.lang.Long) logProps } diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 1831a49124c86..5825ab77ce3fa 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -354,7 +354,7 @@ class LogTest extends JUnitSuite { logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val numMessages = 100 - val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(value = i.toString.getBytes, + val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(value = i.toString.getBytes, timestamp = time.milliseconds)) messageSets.foreach(log.append(_)) log.flush() @@ -389,11 +389,11 @@ class LogTest extends JUnitSuite { val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ - log.append(MemoryRecords.withRecords(CompressionType.GZIP, - Record.create(time.milliseconds, null, "hello".getBytes), + log.append(MemoryRecords.withRecords(CompressionType.GZIP, + Record.create(time.milliseconds, null, "hello".getBytes), Record.create(time.milliseconds, null, "there".getBytes))) - log.append(MemoryRecords.withRecords(CompressionType.GZIP, - Record.create(time.milliseconds, null, "alpha".getBytes), + log.append(MemoryRecords.withRecords(CompressionType.GZIP, + Record.create(time.milliseconds, null, "alpha".getBytes), Record.create(time.milliseconds, null, "beta".getBytes))) def read(offset: Int) = log.read(offset, 4096).records.deepEntries.iterator @@ -445,7 +445,7 @@ class LogTest extends JUnitSuite { */ @Test def testMessageSetSizeCheck() { - val messageSet = MemoryRecords.withRecords(Record.create(time.milliseconds, null, "You".getBytes), + val messageSet = MemoryRecords.withRecords(Record.create(time.milliseconds, null, "You".getBytes), Record.create(time.milliseconds, null, "bethe".getBytes)) // append messages to log val configSegmentSize = messageSet.sizeInBytes - 1 @@ -513,10 +513,10 @@ class LogTest extends JUnitSuite { */ @Test def testMessageSizeCheck() { - val first = MemoryRecords.withRecords(CompressionType.NONE, - Record.create(time.milliseconds, null, "You".getBytes), + val first = MemoryRecords.withRecords(CompressionType.NONE, + Record.create(time.milliseconds, null, "You".getBytes), Record.create(time.milliseconds, null, "bethe".getBytes)) - val second = MemoryRecords.withRecords(CompressionType.NONE, + val second = MemoryRecords.withRecords(CompressionType.NONE, Record.create(time.milliseconds, null, "change (I need more bytes)".getBytes)) // append messages to log @@ -989,6 +989,16 @@ class LogTest extends JUnitSuite { log.append(invalidMessage, assignOffsets = false) } + @Test + def testAppendWithNoTimestamp(): Unit = { + val log = new Log(logDir, + LogConfig(), + recoveryPoint = 0L, + time.scheduler, + time) + log.append(MemoryRecords.withRecords(Record.create(Record.NO_TIMESTAMP, "key".getBytes, "value".getBytes))) + } + @Test def testCorruptLog() { // append some messages to create some segments diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 0f5ff5dec6300..c89e626fa98bc 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -586,6 +586,7 @@ class KafkaConfigTest { case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogMessageTimestampDifferenceMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") @@ -702,6 +703,7 @@ class KafkaConfigTest { assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis) assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis) assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis) + assertEquals(config.logRetentionTimeMillis, config.logMessageTimestampDifferenceMaxMs) assertEquals(123L, config.logFlushIntervalMs) assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec) assertEquals(Sensor.RecordingLevel.DEBUG.toString, config.metricRecordingLevel) diff --git a/docs/upgrade.html b/docs/upgrade.html index 597605461ad5e..ef9216ef195bf 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -15,6 +15,44 @@ limitations under the License. --> +

Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x or 0.10.2.x to 0.10.3.0

+

0.10.2.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. + However, please review the notable changes in 0.10.3.0 before upgrading. +

+ +

Starting with version 0.10.2, Java clients (producer and consumer) have acquired the ability to communicate with older brokers. Version 0.10.3 + clients can talk to version 0.10.0 or newer brokers. However, if your brokers are older than 0.10.0, you must upgrade all the brokers in the + Kafka cluster before upgrading your clients. Version 0.10.3 brokers support 0.8.x and newer clients. +

+ +

For a rolling upgrade:

+ +
    +
  1. Update server.properties file on all brokers and add the following properties: +
      +
    • inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1 or 0.10.2).
    • +
    • log.message.format.version=CURRENT_KAFKA_VERSION (See potential performance impact following the upgrade for the details on what this configuration does.) +
    +
  2. +
  3. Upgrade the brokers one at a time: shut down the broker, update the code, and restart it.
  4. +
  5. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.3.
  6. +
  7. If your previous message format is 0.10.0, change log.message.format.version to 0.10.3 (this is a no-op as the message format is the same for 0.10.0, 0.10.1, 0.10.2 and 0.10.3). + If your previous message format version is lower than 0.10.0, do not change log.message.format.version yet - this parameter should only change once all consumers have been upgraded to 0.10.0.0 or later.
  8. +
  9. Restart the brokers one by one for the new protocol version to take effect.
  10. +
  11. If log.message.format.version is still lower than 0.10.0 at this point, wait until all consumers have been upgraded to 0.10.0 or later, + then change log.message.format.version to 0.10.3 on each broker and restart them one by one.
  12. +
+ +

Note: If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default. + +

Note: Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. + +

Notable changes in 0.10.3.0
+
    +
  • The offsets.topic.replication.factor broker config is now enforced upon auto topic creation. Internal auto topic creation will fail with a GROUP_COORDINATOR_NOT_AVAILABLE error until the cluster size meets this replication factor requirement.
  • +
  • By default message.timestamp.difference.max.ms is the same as retention.ms instead of Long.MAX_VALUE.
  • +
+

Upgrading from 0.8.x, 0.9.x, 0.10.0.x or 0.10.1.x to 0.10.2.0

0.10.2.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade. However, please review the notable changes in 0.10.2.0 before upgrading. @@ -75,7 +113,6 @@

Notable changes in should not be set in the Streams app any more. If the Kafka cluster is secured, Streams apps must have the required security privileges to create new topics.
  • Several new fields including "security.protocol", "connections.max.idle.ms", "retry.backoff.ms", "reconnect.backoff.ms" and "request.timeout.ms" were added to StreamsConfig class. User should pay attention to the default values and set these if needed. For more details please refer to 3.5 Kafka Streams Configs.
  • -
  • The offsets.topic.replication.factor broker config is now enforced upon auto topic creation. Internal auto topic creation will fail with a GROUP_COORDINATOR_NOT_AVAILABLE error until the cluster size meets this replication factor requirement.
  • New Protocol Versions
    From fc1cfe475e8ae8458d8ddf119ce18d0c64653a70 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 17 Feb 2017 14:19:01 -0800 Subject: [PATCH 008/101] MINOR: Remove Struct from Request/Response classes More details: * Replaced `struct` field in Request/Response with a `toStruct` method. This makes the performance model (including memory usage) easier to understand. Note that requests have `toStruct()` while responses have `toStruct(version)`. * Replaced mutable `version` field in `Request.Builder` with an immutable field `desiredVersion` and a `version` parameter passed to the `build` method. * Optimised `handleFetchRequest` to avoid unnecessary creation of `Struct` instances (from 4 to 2 in the worst case and 2 to 1 in the best case). * Various clean-ups in request/response classes and their test. In particular, it is now clear what we are testing. Previously, it looked like we were testing more than we really were. With this in place, we could remove `AbstractRequest.Builder` in the future by doing the following: * Change `AbstractRequest.toStruct` to accept a version (like responses). * Change `AbstractRequest.version` to be `desiredVersion` (like `Builder`). * Change `ClientRequest` to take `AbstractRequest`. * Move validation from the `build` methods to the request constructors or static factory methods. * Anything else required for the code to compile again. Author: Ismael Juma Reviewers: Apurva Mehta , Jason Gustafson Closes #2513 from ijuma/separate-struct --- .../apache/kafka/clients/ClientRequest.java | 5 +- .../apache/kafka/clients/NetworkClient.java | 21 +- .../internals/ConsumerCoordinator.java | 9 +- .../internals/ConsumerNetworkClient.java | 5 +- .../clients/consumer/internals/Fetcher.java | 19 +- .../common/requests/AbstractRequest.java | 110 ++++-- .../requests/AbstractRequestResponse.java | 53 +-- .../common/requests/AbstractResponse.java | 25 +- .../common/requests/ApiVersionsRequest.java | 26 +- .../common/requests/ApiVersionsResponse.java | 49 +-- .../requests/ControlledShutdownRequest.java | 23 +- .../requests/ControlledShutdownResponse.java | 40 +- .../common/requests/CreateTopicsRequest.java | 106 +++--- .../common/requests/CreateTopicsResponse.java | 45 +-- .../common/requests/DeleteTopicsRequest.java | 28 +- .../common/requests/DeleteTopicsResponse.java | 35 +- .../requests/DescribeGroupsRequest.java | 25 +- .../requests/DescribeGroupsResponse.java | 67 ++-- .../kafka/common/requests/FetchRequest.java | 109 +++--- .../kafka/common/requests/FetchResponse.java | 74 ++-- .../requests/GroupCoordinatorRequest.java | 20 +- .../requests/GroupCoordinatorResponse.java | 26 +- .../common/requests/HeartbeatRequest.java | 27 +- .../common/requests/HeartbeatResponse.java | 17 +- .../common/requests/JoinGroupRequest.java | 51 ++- .../common/requests/JoinGroupResponse.java | 57 ++- .../common/requests/LeaderAndIsrRequest.java | 86 ++--- .../common/requests/LeaderAndIsrResponse.java | 46 +-- .../common/requests/LeaveGroupRequest.java | 24 +- .../common/requests/LeaveGroupResponse.java | 22 +- .../common/requests/ListGroupsRequest.java | 19 +- .../common/requests/ListGroupsResponse.java | 33 +- .../common/requests/ListOffsetRequest.java | 113 +++--- .../common/requests/ListOffsetResponse.java | 76 ++-- .../common/requests/MetadataRequest.java | 33 +- .../common/requests/MetadataResponse.java | 129 +++---- .../common/requests/OffsetCommitRequest.java | 135 +++---- .../common/requests/OffsetCommitResponse.java | 52 ++- .../common/requests/OffsetFetchRequest.java | 75 ++-- .../common/requests/OffsetFetchResponse.java | 77 ++-- .../kafka/common/requests/ProduceRequest.java | 70 ++-- .../common/requests/ProduceResponse.java | 42 +-- .../kafka/common/requests/RequestAndSize.java | 27 ++ .../kafka/common/requests/RequestHeader.java | 21 +- .../kafka/common/requests/ResponseHeader.java | 15 +- .../common/requests/SaslHandshakeRequest.java | 21 +- .../requests/SaslHandshakeResponse.java | 21 +- .../common/requests/StopReplicaRequest.java | 50 +-- .../common/requests/StopReplicaResponse.java | 49 ++- .../common/requests/SyncGroupRequest.java | 44 +-- .../common/requests/SyncGroupResponse.java | 21 +- .../requests/UpdateMetadataRequest.java | 130 +++---- .../requests/UpdateMetadataResponse.java | 18 +- .../SaslServerAuthenticator.java | 11 +- .../kafka/common/utils/CollectionUtils.java | 12 +- .../org/apache/kafka/clients/MockClient.java | 17 +- .../kafka/clients/NetworkClientTest.java | 10 +- .../clients/consumer/KafkaConsumerTest.java | 2 +- .../consumer/internals/FetcherTest.java | 2 +- .../common/requests/RequestResponseTest.java | 345 ++++++++++-------- .../authenticator/SaslAuthenticatorTest.java | 2 +- .../main/scala/kafka/api/FetchRequest.scala | 2 +- .../kafka/api/GenericRequestAndHeader.scala | 56 --- .../kafka/api/GenericResponseAndHeader.scala | 47 --- .../scala/kafka/api/ProducerRequest.scala | 2 +- .../controller/ControllerChannelManager.scala | 11 +- .../scala/kafka/network/RequestChannel.scala | 36 +- .../main/scala/kafka/server/KafkaApis.scala | 97 +++-- .../kafka/server/ReplicaFetcherThread.scala | 15 +- .../kafka/api/AuthorizerIntegrationTest.scala | 33 +- .../unit/kafka/network/SocketServerTest.scala | 26 +- .../AbstractCreateTopicsRequestTest.scala | 20 +- .../kafka/server/ApiVersionsRequestTest.scala | 14 +- .../unit/kafka/server/BaseRequestTest.scala | 70 ++-- .../server/CreateTopicsRequestTest.scala | 21 +- .../server/DeleteTopicsRequestTest.scala | 6 +- .../kafka/server/EdgeCaseRequestTest.scala | 20 +- .../unit/kafka/server/FetchRequestTest.scala | 15 +- .../unit/kafka/server/MetadataCacheTest.scala | 32 +- .../kafka/server/MetadataRequestTest.scala | 16 +- .../kafka/server/ProduceRequestTest.scala | 4 +- .../server/SaslApiVersionsRequestTest.scala | 31 +- .../internals/InternalTopicManagerTest.java | 4 +- 83 files changed, 1681 insertions(+), 1819 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java delete mode 100644 core/src/main/scala/kafka/api/GenericRequestAndHeader.scala delete mode 100644 core/src/main/scala/kafka/api/GenericResponseAndHeader.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index 08b8d465a13db..a1973ad0edefd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -74,9 +74,8 @@ public ApiKeys apiKey() { return requestBuilder.apiKey(); } - public RequestHeader makeHeader() { - return new RequestHeader(requestBuilder.apiKey().id, - requestBuilder.version(), clientId, correlationId); + public RequestHeader makeHeader(short version) { + return new RequestHeader(requestBuilder.apiKey().id, version, clientId, correlationId); } public AbstractRequest.Builder requestBuilder() { diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 890bf56d1293e..4131bcb41d9a3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -280,43 +280,46 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long if (!canSendRequest(nodeId)) throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); } - AbstractRequest request = null; AbstractRequest.Builder builder = clientRequest.requestBuilder(); try { NodeApiVersions versionInfo = nodeApiVersions.get(nodeId); + short version; // Note: if versionInfo is null, we have no server version information. This would be // the case when sending the initial ApiVersionRequest which fetches the version // information itself. It is also the case when discoverBrokerVersions is set to false. if (versionInfo == null) { + version = builder.desiredOrLatestVersion(); if (discoverBrokerVersions && log.isTraceEnabled()) log.trace("No version information found when sending message of type {} to node {}. " + - "Assuming version {}.", clientRequest.apiKey(), nodeId, builder.version()); + "Assuming version {}.", clientRequest.apiKey(), nodeId, version); } else { - short version = versionInfo.usableVersion(clientRequest.apiKey()); - builder.setVersion(version); + version = versionInfo.usableVersion(clientRequest.apiKey()); } // The call to build may also throw UnsupportedVersionException, if there are essential // fields that cannot be represented in the chosen version. - request = builder.build(); + doSend(clientRequest, isInternalRequest, now, builder.build(version)); } catch (UnsupportedVersionException e) { // If the version is not supported, skip sending the request over the wire. // Instead, simply add it to the local queue of aborted requests. log.debug("Version mismatch when attempting to send {} to {}", clientRequest.toString(), clientRequest.destination(), e); - ClientResponse clientResponse = new ClientResponse(clientRequest.makeHeader(), + ClientResponse clientResponse = new ClientResponse(clientRequest.makeHeader(builder.desiredOrLatestVersion()), clientRequest.callback(), clientRequest.destination(), now, now, false, e, null); abortedSends.add(clientResponse); - return; } - RequestHeader header = clientRequest.makeHeader(); + } + + private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long now, AbstractRequest request) { + String nodeId = clientRequest.destination(); + RequestHeader header = clientRequest.makeHeader(request.version()); if (log.isDebugEnabled()) { int latestClientVersion = ProtoUtils.latestVersion(clientRequest.apiKey().id); if (header.apiVersion() == latestClientVersion) { log.trace("Sending {} to node {}.", request, nodeId); } else { log.debug("Using older server API v{} to send {} to node {}.", - header.apiVersion(), request, nodeId); + header.apiVersion(), request, nodeId); } } Send send = request.toSend(nodeId, header); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 0dc073e21f6e4..12ff9ce699113 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -697,11 +697,10 @@ private RequestFuture sendOffsetCommitRequest(final Map> sendListOffsetRequest(final Node node, - final Map timestampsToSearch, - boolean requireTimestamp) { - ListOffsetRequest.Builder builder = new ListOffsetRequest.Builder().setTargetTimes(timestampsToSearch); - - // If we need a timestamp in the response, the minimum RPC version we can send is v1. - // Otherwise, v0 is OK. - builder.setMinVersion(requireTimestamp ? (short) 1 : (short) 0); + final Map timestampsToSearch, + boolean requireTimestamp) { + // If we need a timestamp in the response, the minimum RPC version we can send is v1. Otherwise, v0 is OK. + short minVersion = requireTimestamp ? (short) 1 : (short) 0; + ListOffsetRequest.Builder builder = ListOffsetRequest.Builder.forConsumer(minVersion) + .setTargetTimes(timestampsToSearch); log.trace("Sending ListOffsetRequest {} to broker {}", builder, node); return client.send(node, builder) @@ -733,7 +732,7 @@ private Map createFetchRequests() { Map requests = new HashMap<>(); for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); - FetchRequest.Builder fetch = new FetchRequest.Builder(this.maxWaitMs, this.minBytes, entry.getValue()). + FetchRequest.Builder fetch = FetchRequest.Builder.forConsumer(this.maxWaitMs, this.minBytes, entry.getValue()). setMaxBytes(this.maxBytes); requests.put(node, fetch); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index f2ea4209355ec..eea1916cb7710 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -25,40 +25,39 @@ import java.nio.ByteBuffer; public abstract class AbstractRequest extends AbstractRequestResponse { - private final short version; public static abstract class Builder { private final ApiKeys apiKey; - private short version; + private final Short desiredVersion; public Builder(ApiKeys apiKey) { + this(apiKey, null); + } + + public Builder(ApiKeys apiKey, Short desiredVersion) { this.apiKey = apiKey; - this.version = ProtoUtils.latestVersion(apiKey.id); + this.desiredVersion = desiredVersion; } public ApiKeys apiKey() { return apiKey; } - public Builder setVersion(short version) { - this.version = version; - return this; + public short desiredOrLatestVersion() { + return desiredVersion == null ? ProtoUtils.latestVersion(apiKey.id) : desiredVersion; } - public short version() { - return version; + public T build() { + return build(desiredOrLatestVersion()); } - public abstract T build(); + public abstract T build(short version); } - public AbstractRequest(Struct struct, short version) { - super(struct); - this.version = version; - } + private final short version; - public Send toSend(String destination, RequestHeader header) { - return new NetworkSend(destination, serialize(header, this)); + public AbstractRequest(short version) { + this.version = version; } /** @@ -68,6 +67,19 @@ public short version() { return version; } + public Send toSend(String destination, RequestHeader header) { + return new NetworkSend(destination, serialize(header)); + } + + /** + * Use with care, typically {@link #toSend(String, RequestHeader)} should be used instead. + */ + public ByteBuffer serialize(RequestHeader header) { + return serialize(header.toStruct(), toStruct()); + } + + protected abstract Struct toStruct(); + /** * Get an error response for a request */ @@ -76,54 +88,78 @@ public short version() { /** * Factory method for getting a request object based on ApiKey ID and a buffer */ - public static AbstractRequest getRequest(int requestId, short versionId, ByteBuffer buffer) { + public static RequestAndSize getRequest(int requestId, short version, ByteBuffer buffer) { ApiKeys apiKey = ApiKeys.forId(requestId); + Struct struct = ProtoUtils.parseRequest(apiKey.id, version, buffer); + AbstractRequest request; switch (apiKey) { case PRODUCE: - return ProduceRequest.parse(buffer, versionId); + request = new ProduceRequest(struct, version); + break; case FETCH: - return FetchRequest.parse(buffer, versionId); + request = new FetchRequest(struct, version); + break; case LIST_OFFSETS: - return ListOffsetRequest.parse(buffer, versionId); + request = new ListOffsetRequest(struct, version); + break; case METADATA: - return MetadataRequest.parse(buffer, versionId); + request = new MetadataRequest(struct, version); + break; case OFFSET_COMMIT: - return OffsetCommitRequest.parse(buffer, versionId); + request = new OffsetCommitRequest(struct, version); + break; case OFFSET_FETCH: - return OffsetFetchRequest.parse(buffer, versionId); + request = new OffsetFetchRequest(struct, version); + break; case GROUP_COORDINATOR: - return GroupCoordinatorRequest.parse(buffer, versionId); + request = new GroupCoordinatorRequest(struct, version); + break; case JOIN_GROUP: - return JoinGroupRequest.parse(buffer, versionId); + request = new JoinGroupRequest(struct, version); + break; case HEARTBEAT: - return HeartbeatRequest.parse(buffer, versionId); + request = new HeartbeatRequest(struct, version); + break; case LEAVE_GROUP: - return LeaveGroupRequest.parse(buffer, versionId); + request = new LeaveGroupRequest(struct, version); + break; case SYNC_GROUP: - return SyncGroupRequest.parse(buffer, versionId); + request = new SyncGroupRequest(struct, version); + break; case STOP_REPLICA: - return StopReplicaRequest.parse(buffer, versionId); + request = new StopReplicaRequest(struct, version); + break; case CONTROLLED_SHUTDOWN_KEY: - return ControlledShutdownRequest.parse(buffer, versionId); + request = new ControlledShutdownRequest(struct, version); + break; case UPDATE_METADATA_KEY: - return UpdateMetadataRequest.parse(buffer, versionId); + request = new UpdateMetadataRequest(struct, version); + break; case LEADER_AND_ISR: - return LeaderAndIsrRequest.parse(buffer, versionId); + request = new LeaderAndIsrRequest(struct, version); + break; case DESCRIBE_GROUPS: - return DescribeGroupsRequest.parse(buffer, versionId); + request = new DescribeGroupsRequest(struct, version); + break; case LIST_GROUPS: - return ListGroupsRequest.parse(buffer, versionId); + request = new ListGroupsRequest(struct, version); + break; case SASL_HANDSHAKE: - return SaslHandshakeRequest.parse(buffer, versionId); + request = new SaslHandshakeRequest(struct, version); + break; case API_VERSIONS: - return ApiVersionsRequest.parse(buffer, versionId); + request = new ApiVersionsRequest(struct, version); + break; case CREATE_TOPICS: - return CreateTopicsRequest.parse(buffer, versionId); + request = new CreateTopicsRequest(struct, version); + break; case DELETE_TOPICS: - return DeleteTopicsRequest.parse(buffer, versionId); + request = new DeleteTopicsRequest(struct, version); + break; default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `getRequest`, the " + "code should be updated to do so.", apiKey)); } + return new RequestAndSize(request, struct.sizeOf()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java index 3ad16a54575c4..00ddf71dc5737 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java @@ -17,56 +17,13 @@ import java.nio.ByteBuffer; public abstract class AbstractRequestResponse { - protected final Struct struct; - - public AbstractRequestResponse(Struct struct) { - this.struct = struct; - } - - public Struct toStruct() { - return struct; - } - - /** - * Get the serialized size of this object - */ - public int sizeOf() { - return struct.sizeOf(); - } - /** - * Write this object to a buffer + * Visible for testing. */ - public void writeTo(ByteBuffer buffer) { - struct.writeTo(buffer); - } - - @Override - public String toString() { - return struct.toString(); - } - - @Override - public int hashCode() { - return struct.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - AbstractRequestResponse other = (AbstractRequestResponse) obj; - return struct.equals(other.struct); - } - - public static ByteBuffer serialize(AbstractRequestResponse header, AbstractRequestResponse body) { - ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf()); - header.writeTo(buffer); - body.writeTo(buffer); + public static ByteBuffer serialize(Struct headerStruct, Struct bodyStruct) { + ByteBuffer buffer = ByteBuffer.allocate(headerStruct.sizeOf() + bodyStruct.sizeOf()); + headerStruct.writeTo(buffer); + bodyStruct.writeTo(buffer); buffer.rewind(); return buffer; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 8bbc25a40e3c5..a21e340080c7d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -21,17 +21,32 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.types.Struct; +import java.nio.ByteBuffer; + public abstract class AbstractResponse extends AbstractRequestResponse { - public AbstractResponse(Struct struct) { - super(struct); + public Send toSend(String destination, RequestHeader requestHeader) { + return toSend(destination, requestHeader.apiVersion(), requestHeader.toResponseHeader()); + } + + /** + * This should only be used if we need to return a response with a different version than the request, which + * should be very rare (an example is @link {@link ApiVersionsResponse#unsupportedVersionSend(String, RequestHeader)}). + * Typically {@link #toSend(String, RequestHeader)} should be used. + */ + public Send toSend(String destination, short version, ResponseHeader responseHeader) { + return new NetworkSend(destination, serialize(version, responseHeader)); } - public Send toSend(String destination, RequestHeader request) { - ResponseHeader responseHeader = new ResponseHeader(request.correlationId()); - return new NetworkSend(destination, serialize(responseHeader, this)); + /** + * Visible for testing, typically {@link #toSend(String, RequestHeader)} should be used instead. + */ + public ByteBuffer serialize(short version, ResponseHeader responseHeader) { + return serialize(responseHeader.toStruct(), toStruct(version)); } + protected abstract Struct toStruct(short version); + public static AbstractResponse getResponse(int requestId, Struct struct) { ApiKeys apiKey = ApiKeys.forId(requestId); switch (apiKey) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java index fe7c3488a3dc7..7d409007a9cf3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java @@ -22,13 +22,14 @@ public class ApiVersionsRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { + public Builder() { super(ApiKeys.API_VERSIONS); } @Override - public ApiVersionsRequest build() { - return new ApiVersionsRequest(version()); + public ApiVersionsRequest build(short version) { + return new ApiVersionsRequest(version); } @Override @@ -38,12 +39,16 @@ public String toString() { } public ApiVersionsRequest(short version) { - this(new Struct(ProtoUtils.requestSchema(ApiKeys.API_VERSIONS.id, version)), - version); + super(version); } - public ApiVersionsRequest(Struct struct, short versionId) { - super(struct, versionId); + public ApiVersionsRequest(Struct struct, short version) { + super(version); + } + + @Override + protected Struct toStruct() { + return new Struct(ProtoUtils.requestSchema(ApiKeys.API_VERSIONS.id, version())); } @Override @@ -58,13 +63,8 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static ApiVersionsRequest parse(ByteBuffer buffer, int versionId) { - return new ApiVersionsRequest( - ProtoUtils.parseRequest(ApiKeys.API_VERSIONS.id, versionId, buffer), - (short) versionId); + public static ApiVersionsRequest parse(ByteBuffer buffer, short version) { + return new ApiVersionsRequest(ProtoUtils.parseRequest(ApiKeys.API_VERSIONS.id, version, buffer), version); } - public static ApiVersionsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.API_VERSIONS.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 7d8bcc54efbc8..0066c084467e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -12,10 +12,10 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -28,8 +28,6 @@ public class ApiVersionsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.API_VERSIONS.id); - public static final ApiVersionsResponse API_VERSIONS_RESPONSE = createApiVersionsResponse(); public static final String ERROR_CODE_KEY_NAME = "error_code"; public static final String API_VERSIONS_KEY_NAME = "api_versions"; @@ -67,23 +65,11 @@ public String toString() { } public ApiVersionsResponse(Errors error, List apiVersions) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - List apiVersionList = new ArrayList<>(); - for (ApiVersion apiVersion : apiVersions) { - Struct apiVersionStruct = struct.instance(API_VERSIONS_KEY_NAME); - apiVersionStruct.set(API_KEY_NAME, apiVersion.apiKey); - apiVersionStruct.set(MIN_VERSION_KEY_NAME, apiVersion.minVersion); - apiVersionStruct.set(MAX_VERSION_KEY_NAME, apiVersion.maxVersion); - apiVersionList.add(apiVersionStruct); - } - struct.set(API_VERSIONS_KEY_NAME, apiVersionList.toArray()); this.error = error; this.apiKeyToApiVersion = buildApiKeyToApiVersion(apiVersions); } public ApiVersionsResponse(Struct struct) { - super(struct); this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); List tempApiVersions = new ArrayList<>(); for (Object apiVersionsObj : struct.getArray(API_VERSIONS_KEY_NAME)) { @@ -96,6 +82,31 @@ public ApiVersionsResponse(Struct struct) { this.apiKeyToApiVersion = buildApiKeyToApiVersion(tempApiVersions); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.API_VERSIONS.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + List apiVersionList = new ArrayList<>(); + for (ApiVersion apiVersion : apiKeyToApiVersion.values()) { + Struct apiVersionStruct = struct.instance(API_VERSIONS_KEY_NAME); + apiVersionStruct.set(API_KEY_NAME, apiVersion.apiKey); + apiVersionStruct.set(MIN_VERSION_KEY_NAME, apiVersion.minVersion); + apiVersionStruct.set(MAX_VERSION_KEY_NAME, apiVersion.maxVersion); + apiVersionList.add(apiVersionStruct); + } + struct.set(API_VERSIONS_KEY_NAME, apiVersionList.toArray()); + return struct; + } + + /** + * Returns Errors.UNSUPPORTED_VERSION response with version 0 since we don't support the requested version. + */ + public static Send unsupportedVersionSend(String destination, RequestHeader requestHeader) { + ApiVersionsResponse response = new ApiVersionsResponse(Errors.UNSUPPORTED_VERSION, + Collections.emptyList()); + return response.toSend(destination, (short) 0, requestHeader.toResponseHeader()); + } + public Collection apiVersions() { return apiKeyToApiVersion.values(); } @@ -108,12 +119,8 @@ public Errors error() { return error; } - public static ApiVersionsResponse parse(ByteBuffer buffer) { - return new ApiVersionsResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static ApiVersionsResponse fromError(Errors error) { - return new ApiVersionsResponse(error, Collections.emptyList()); + public static ApiVersionsResponse parse(ByteBuffer buffer, short version) { + return new ApiVersionsResponse(ProtoUtils.responseSchema(ApiKeys.API_VERSIONS.id, version).read(buffer)); } private static ApiVersionsResponse createApiVersionsResponse() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java index 8f44e5cfe7c92..679e5dd327958 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java @@ -33,8 +33,8 @@ public Builder(int brokerId) { } @Override - public ControlledShutdownRequest build() { - return new ControlledShutdownRequest(brokerId, version()); + public ControlledShutdownRequest build(short version) { + return new ControlledShutdownRequest(brokerId, version); } @Override @@ -49,14 +49,12 @@ public String toString() { private int brokerId; private ControlledShutdownRequest(int brokerId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version)), - version); - struct.set(BROKER_ID_KEY_NAME, brokerId); + super(version); this.brokerId = brokerId; } - public ControlledShutdownRequest(Struct struct, short versionId) { - super(struct, versionId); + public ControlledShutdownRequest(Struct struct, short version) { + super(version); brokerId = struct.getInt(BROKER_ID_KEY_NAME); } @@ -79,12 +77,15 @@ public int brokerId() { return brokerId; } - public static ControlledShutdownRequest parse(ByteBuffer buffer, int versionId) { + public static ControlledShutdownRequest parse(ByteBuffer buffer, short versionId) { return new ControlledShutdownRequest( - ProtoUtils.parseRequest(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, versionId, buffer), (short) versionId); + ProtoUtils.parseRequest(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, versionId, buffer), versionId); } - public static ControlledShutdownRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version())); + struct.set(BROKER_ID_KEY_NAME, brokerId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java index b3922f9573542..a2cac6c9f096f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,8 +26,6 @@ public class ControlledShutdownResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id); - private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String PARTITIONS_REMAINING_KEY_NAME = "partitions_remaining"; @@ -47,25 +44,11 @@ public class ControlledShutdownResponse extends AbstractResponse { private final Set partitionsRemaining; public ControlledShutdownResponse(Errors error, Set partitionsRemaining) { - super(new Struct(CURRENT_SCHEMA)); - - struct.set(ERROR_CODE_KEY_NAME, error.code()); - - List partitionsRemainingList = new ArrayList<>(partitionsRemaining.size()); - for (TopicPartition topicPartition : partitionsRemaining) { - Struct topicPartitionStruct = struct.instance(PARTITIONS_REMAINING_KEY_NAME); - topicPartitionStruct.set(TOPIC_KEY_NAME, topicPartition.topic()); - topicPartitionStruct.set(PARTITION_KEY_NAME, topicPartition.partition()); - partitionsRemainingList.add(topicPartitionStruct); - } - struct.set(PARTITIONS_REMAINING_KEY_NAME, partitionsRemainingList.toArray()); - this.error = error; this.partitionsRemaining = partitionsRemaining; } public ControlledShutdownResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); Set partitions = new HashSet<>(); for (Object topicPartitionObj : struct.getArray(PARTITIONS_REMAINING_KEY_NAME)) { @@ -85,12 +68,25 @@ public Set partitionsRemaining() { return partitionsRemaining; } - public static ControlledShutdownResponse parse(ByteBuffer buffer) { - return new ControlledShutdownResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static ControlledShutdownResponse parse(ByteBuffer buffer, int version) { + public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) { return new ControlledShutdownResponse(ProtoUtils.parseResponse(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version, buffer)); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version)); + + struct.set(ERROR_CODE_KEY_NAME, error.code()); + + List partitionsRemainingList = new ArrayList<>(partitionsRemaining.size()); + for (TopicPartition topicPartition : partitionsRemaining) { + Struct topicPartitionStruct = struct.instance(PARTITIONS_REMAINING_KEY_NAME); + topicPartitionStruct.set(TOPIC_KEY_NAME, topicPartition.topic()); + topicPartitionStruct.set(PARTITION_KEY_NAME, topicPartition.partition()); + partitionsRemainingList.add(topicPartitionStruct); + } + struct.set(PARTITIONS_REMAINING_KEY_NAME, partitionsRemainingList.toArray()); + + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java index 0a4bce01394ac..a8f8c5e9ece31 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java @@ -113,11 +113,11 @@ public Builder(Map topics, int timeout, boolean validateOn } @Override - public CreateTopicsRequest build() { - if (validateOnly && version() == 0) + public CreateTopicsRequest build(short version) { + if (validateOnly && version == 0) throw new UnsupportedVersionException("validateOnly is not supported in version 0 of " + "CreateTopicsRequest"); - return new CreateTopicsRequest(topics, timeout, validateOnly, version()); + return new CreateTopicsRequest(topics, timeout, validateOnly, version); } @Override @@ -144,53 +144,15 @@ public String toString() { public static final short NO_REPLICATION_FACTOR = -1; private CreateTopicsRequest(Map topics, Integer timeout, boolean validateOnly, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.CREATE_TOPICS.id, version)), version); - - List createTopicRequestStructs = new ArrayList<>(topics.size()); - for (Map.Entry entry : topics.entrySet()) { - - Struct singleRequestStruct = struct.instance(REQUESTS_KEY_NAME); - String topic = entry.getKey(); - TopicDetails args = entry.getValue(); - - singleRequestStruct.set(TOPIC_KEY_NAME, topic); - singleRequestStruct.set(NUM_PARTITIONS_KEY_NAME, args.numPartitions); - singleRequestStruct.set(REPLICATION_FACTOR_KEY_NAME, args.replicationFactor); - - // replica assignment - List replicaAssignmentsStructs = new ArrayList<>(args.replicasAssignments.size()); - for (Map.Entry> partitionReplicaAssignment : args.replicasAssignments.entrySet()) { - Struct replicaAssignmentStruct = singleRequestStruct.instance(REPLICA_ASSIGNMENT_KEY_NAME); - replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_PARTITION_ID_KEY_NAME, partitionReplicaAssignment.getKey()); - replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME, partitionReplicaAssignment.getValue().toArray()); - replicaAssignmentsStructs.add(replicaAssignmentStruct); - } - singleRequestStruct.set(REPLICA_ASSIGNMENT_KEY_NAME, replicaAssignmentsStructs.toArray()); - - // configs - List configsStructs = new ArrayList<>(args.configs.size()); - for (Map.Entry configEntry : args.configs.entrySet()) { - Struct configStruct = singleRequestStruct.instance(CONFIGS_KEY_NAME); - configStruct.set(CONFIG_KEY_KEY_NAME, configEntry.getKey()); - configStruct.set(CONFIG_VALUE_KEY_NAME, configEntry.getValue()); - configsStructs.add(configStruct); - } - singleRequestStruct.set(CONFIGS_KEY_NAME, configsStructs.toArray()); - createTopicRequestStructs.add(singleRequestStruct); - } - struct.set(REQUESTS_KEY_NAME, createTopicRequestStructs.toArray()); - struct.set(TIMEOUT_KEY_NAME, timeout); - if (version >= 1) - struct.set(VALIDATE_ONLY_KEY_NAME, validateOnly); - + super(version); this.topics = topics; this.timeout = timeout; this.validateOnly = validateOnly; this.duplicateTopics = Collections.emptySet(); } - public CreateTopicsRequest(Struct struct, short versionId) { - super(struct, versionId); + public CreateTopicsRequest(Struct struct, short version) { + super(version); Object[] requestStructs = struct.getArray(REQUESTS_KEY_NAME); Map topics = new HashMap<>(); @@ -262,7 +224,7 @@ public AbstractResponse getErrorResponse(Throwable e) { switch (versionId) { case 0: case 1: - return new CreateTopicsResponse(topicErrors, versionId); + return new CreateTopicsResponse(topicErrors); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CREATE_TOPICS.id))); @@ -285,13 +247,55 @@ public Set duplicateTopics() { return this.duplicateTopics; } - public static CreateTopicsRequest parse(ByteBuffer buffer, int versionId) { - return new CreateTopicsRequest( - ProtoUtils.parseRequest(ApiKeys.CREATE_TOPICS.id, versionId, buffer), - (short) versionId); + public static CreateTopicsRequest parse(ByteBuffer buffer, short versionId) { + return new CreateTopicsRequest(ProtoUtils.parseRequest(ApiKeys.CREATE_TOPICS.id, versionId, buffer), versionId); } - public static CreateTopicsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.CREATE_TOPICS.id)); + /** + * Visible for testing. + */ + @Override + public Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.CREATE_TOPICS.id, version)); + + List createTopicRequestStructs = new ArrayList<>(topics.size()); + for (Map.Entry entry : topics.entrySet()) { + + Struct singleRequestStruct = struct.instance(REQUESTS_KEY_NAME); + String topic = entry.getKey(); + TopicDetails args = entry.getValue(); + + singleRequestStruct.set(TOPIC_KEY_NAME, topic); + singleRequestStruct.set(NUM_PARTITIONS_KEY_NAME, args.numPartitions); + singleRequestStruct.set(REPLICATION_FACTOR_KEY_NAME, args.replicationFactor); + + // replica assignment + List replicaAssignmentsStructs = new ArrayList<>(args.replicasAssignments.size()); + for (Map.Entry> partitionReplicaAssignment : args.replicasAssignments.entrySet()) { + Struct replicaAssignmentStruct = singleRequestStruct.instance(REPLICA_ASSIGNMENT_KEY_NAME); + replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_PARTITION_ID_KEY_NAME, partitionReplicaAssignment.getKey()); + replicaAssignmentStruct.set(REPLICA_ASSIGNMENT_REPLICAS_KEY_NAME, partitionReplicaAssignment.getValue().toArray()); + replicaAssignmentsStructs.add(replicaAssignmentStruct); + } + singleRequestStruct.set(REPLICA_ASSIGNMENT_KEY_NAME, replicaAssignmentsStructs.toArray()); + + // configs + List configsStructs = new ArrayList<>(args.configs.size()); + for (Map.Entry configEntry : args.configs.entrySet()) { + Struct configStruct = singleRequestStruct.instance(CONFIGS_KEY_NAME); + configStruct.set(CONFIG_KEY_KEY_NAME, configEntry.getKey()); + configStruct.set(CONFIG_VALUE_KEY_NAME, configEntry.getValue()); + configsStructs.add(configStruct); + } + singleRequestStruct.set(CONFIGS_KEY_NAME, configsStructs.toArray()); + createTopicRequestStructs.add(singleRequestStruct); + } + struct.set(REQUESTS_KEY_NAME, createTopicRequestStructs.toArray()); + struct.set(TIMEOUT_KEY_NAME, timeout); + if (version >= 1) + struct.set(VALIDATE_ONLY_KEY_NAME, validateOnly); + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java index b09795f3d32cc..01b7c2b17d577 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -30,8 +29,6 @@ import java.util.Map; public class CreateTopicsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CREATE_TOPICS.id); - private static final String TOPIC_ERRORS_KEY_NAME = "topic_errors"; private static final String TOPIC_KEY_NAME = "topic"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -87,27 +84,11 @@ public String toString() { private final Map errors; - public CreateTopicsResponse(Map errors, short version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version))); - - List topicErrorsStructs = new ArrayList<>(errors.size()); - for (Map.Entry topicError : errors.entrySet()) { - Struct topicErrorsStruct = struct.instance(TOPIC_ERRORS_KEY_NAME); - topicErrorsStruct.set(TOPIC_KEY_NAME, topicError.getKey()); - Error error = topicError.getValue(); - topicErrorsStruct.set(ERROR_CODE_KEY_NAME, error.error.code()); - if (version >= 1) - topicErrorsStruct.set(ERROR_MESSAGE_KEY_NAME, error.message()); - topicErrorsStructs.add(topicErrorsStruct); - } - struct.set(TOPIC_ERRORS_KEY_NAME, topicErrorsStructs.toArray()); - + public CreateTopicsResponse(Map errors) { this.errors = errors; } public CreateTopicsResponse(Struct struct) { - super(struct); - Object[] topicErrorStructs = struct.getArray(TOPIC_ERRORS_KEY_NAME); Map errors = new HashMap<>(); for (Object topicErrorStructObj : topicErrorStructs) { @@ -123,15 +104,29 @@ public CreateTopicsResponse(Struct struct) { this.errors = errors; } - public Map errors() { - return errors; + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version)); + + List topicErrorsStructs = new ArrayList<>(errors.size()); + for (Map.Entry topicError : errors.entrySet()) { + Struct topicErrorsStruct = struct.instance(TOPIC_ERRORS_KEY_NAME); + topicErrorsStruct.set(TOPIC_KEY_NAME, topicError.getKey()); + Error error = topicError.getValue(); + topicErrorsStruct.set(ERROR_CODE_KEY_NAME, error.error.code()); + if (version >= 1) + topicErrorsStruct.set(ERROR_MESSAGE_KEY_NAME, error.message()); + topicErrorsStructs.add(topicErrorsStruct); + } + struct.set(TOPIC_ERRORS_KEY_NAME, topicErrorsStructs.toArray()); + return struct; } - public static CreateTopicsResponse parse(ByteBuffer buffer) { - return new CreateTopicsResponse(CURRENT_SCHEMA.read(buffer)); + public Map errors() { + return errors; } - public static CreateTopicsResponse parse(ByteBuffer buffer, int version) { + public static CreateTopicsResponse parse(ByteBuffer buffer, short version) { return new CreateTopicsResponse(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java index 2874fadc7a99b..eea4aa93cd60d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java @@ -46,8 +46,8 @@ public Builder(Set topics, Integer timeout) { } @Override - public DeleteTopicsRequest build() { - return new DeleteTopicsRequest(topics, timeout, version()); + public DeleteTopicsRequest build(short version) { + return new DeleteTopicsRequest(topics, timeout, version); } @Override @@ -62,17 +62,13 @@ public String toString() { } private DeleteTopicsRequest(Set topics, Integer timeout, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.DELETE_TOPICS.id, version)), version); - - struct.set(TOPICS_KEY_NAME, topics.toArray()); - struct.set(TIMEOUT_KEY_NAME, timeout); - + super(version); this.topics = topics; this.timeout = timeout; } public DeleteTopicsRequest(Struct struct, short version) { - super(struct, version); + super(version); Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME); Set topics = new HashSet<>(topicsArray.length); for (Object topic : topicsArray) @@ -82,6 +78,14 @@ public DeleteTopicsRequest(Struct struct, short version) { this.timeout = struct.getInt(TIMEOUT_KEY_NAME); } + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.DELETE_TOPICS.id, version())); + struct.set(TOPICS_KEY_NAME, topics.toArray()); + struct.set(TIMEOUT_KEY_NAME, timeout); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { Map topicErrors = new HashMap<>(); @@ -105,12 +109,8 @@ public Integer timeout() { return this.timeout; } - public static DeleteTopicsRequest parse(ByteBuffer buffer, int versionId) { - return new DeleteTopicsRequest(ProtoUtils.parseRequest(ApiKeys.DELETE_TOPICS.id, versionId, buffer), - (short) versionId); + public static DeleteTopicsRequest parse(ByteBuffer buffer, short versionId) { + return new DeleteTopicsRequest(ProtoUtils.parseRequest(ApiKeys.DELETE_TOPICS.id, versionId, buffer), versionId); } - public static DeleteTopicsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java index 5c8b3d5a5dc53..c47d0982b248c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -29,7 +28,6 @@ import java.util.Map; public class DeleteTopicsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.DELETE_TOPICS.id); private static final String TOPIC_ERROR_CODES_KEY_NAME = "topic_error_codes"; private static final String TOPIC_KEY_NAME = "topic"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -45,23 +43,10 @@ public class DeleteTopicsResponse extends AbstractResponse { private final Map errors; public DeleteTopicsResponse(Map errors) { - super(new Struct(CURRENT_SCHEMA)); - - List topicErrorCodeStructs = new ArrayList<>(errors.size()); - for (Map.Entry topicError : errors.entrySet()) { - Struct topicErrorCodeStruct = struct.instance(TOPIC_ERROR_CODES_KEY_NAME); - topicErrorCodeStruct.set(TOPIC_KEY_NAME, topicError.getKey()); - topicErrorCodeStruct.set(ERROR_CODE_KEY_NAME, topicError.getValue().code()); - topicErrorCodeStructs.add(topicErrorCodeStruct); - } - struct.set(TOPIC_ERROR_CODES_KEY_NAME, topicErrorCodeStructs.toArray()); - this.errors = errors; } public DeleteTopicsResponse(Struct struct) { - super(struct); - Object[] topicErrorCodesStructs = struct.getArray(TOPIC_ERROR_CODES_KEY_NAME); Map errors = new HashMap<>(); for (Object topicErrorCodeStructObj : topicErrorCodesStructs) { @@ -74,15 +59,25 @@ public DeleteTopicsResponse(Struct struct) { this.errors = errors; } - public Map errors() { - return errors; + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.DELETE_TOPICS.id, version)); + List topicErrorCodeStructs = new ArrayList<>(errors.size()); + for (Map.Entry topicError : errors.entrySet()) { + Struct topicErrorCodeStruct = struct.instance(TOPIC_ERROR_CODES_KEY_NAME); + topicErrorCodeStruct.set(TOPIC_KEY_NAME, topicError.getKey()); + topicErrorCodeStruct.set(ERROR_CODE_KEY_NAME, topicError.getValue().code()); + topicErrorCodeStructs.add(topicErrorCodeStruct); + } + struct.set(TOPIC_ERROR_CODES_KEY_NAME, topicErrorCodeStructs.toArray()); + return struct; } - public static DeleteTopicsResponse parse(ByteBuffer buffer) { - return new DeleteTopicsResponse(CURRENT_SCHEMA.read(buffer)); + public Map errors() { + return errors; } - public static DeleteTopicsResponse parse(ByteBuffer buffer, int version) { + public static DeleteTopicsResponse parse(ByteBuffer buffer, short version) { return new DeleteTopicsResponse(ProtoUtils.responseSchema(ApiKeys.DELETE_TOPICS.id, version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java index f17cdd90b3021..0f1337101dc81 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java @@ -34,8 +34,7 @@ public Builder(List groupIds) { } @Override - public DescribeGroupsRequest build() { - short version = version(); + public DescribeGroupsRequest build(short version) { return new DescribeGroupsRequest(this.groupIds, version); } @@ -48,14 +47,12 @@ public String toString() { private final List groupIds; private DescribeGroupsRequest(List groupIds, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.DESCRIBE_GROUPS.id, version)), - version); - struct.set(GROUP_IDS_KEY_NAME, groupIds.toArray()); + super(version); this.groupIds = groupIds; } public DescribeGroupsRequest(Struct struct, short version) { - super(struct, version); + super(version); this.groupIds = new ArrayList<>(); for (Object groupId : struct.getArray(GROUP_IDS_KEY_NAME)) this.groupIds.add((String) groupId); @@ -65,6 +62,13 @@ public List groupIds() { return groupIds; } + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.DESCRIBE_GROUPS.id, version())); + struct.set(GROUP_IDS_KEY_NAME, groupIds.toArray()); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { short versionId = version(); @@ -78,12 +82,7 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static DescribeGroupsRequest parse(ByteBuffer buffer, int versionId) { - return new DescribeGroupsRequest(ProtoUtils.parseRequest(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer), - (short) versionId); - } - - public static DescribeGroupsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.DESCRIBE_GROUPS.id)); + public static DescribeGroupsRequest parse(ByteBuffer buffer, short versionId) { + return new DescribeGroupsRequest(ProtoUtils.parseRequest(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer), versionId); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java index 56b387e37496c..0dde987476de8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,8 +26,6 @@ public class DescribeGroupsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.DESCRIBE_GROUPS.id); - private static final String GROUPS_KEY_NAME = "groups"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -60,36 +57,10 @@ public class DescribeGroupsResponse extends AbstractResponse { private final Map groups; public DescribeGroupsResponse(Map groups) { - super(new Struct(CURRENT_SCHEMA)); - - List groupStructs = new ArrayList<>(); - for (Map.Entry groupEntry : groups.entrySet()) { - Struct groupStruct = struct.instance(GROUPS_KEY_NAME); - GroupMetadata group = groupEntry.getValue(); - groupStruct.set(GROUP_ID_KEY_NAME, groupEntry.getKey()); - groupStruct.set(ERROR_CODE_KEY_NAME, group.error.code()); - groupStruct.set(GROUP_STATE_KEY_NAME, group.state); - groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); - groupStruct.set(PROTOCOL_KEY_NAME, group.protocol); - List membersList = new ArrayList<>(); - for (GroupMember member : group.members) { - Struct memberStruct = groupStruct.instance(MEMBERS_KEY_NAME); - memberStruct.set(MEMBER_ID_KEY_NAME, member.memberId); - memberStruct.set(CLIENT_ID_KEY_NAME, member.clientId); - memberStruct.set(CLIENT_HOST_KEY_NAME, member.clientHost); - memberStruct.set(MEMBER_METADATA_KEY_NAME, member.memberMetadata); - memberStruct.set(MEMBER_ASSIGNMENT_KEY_NAME, member.memberAssignment); - membersList.add(memberStruct); - } - groupStruct.set(MEMBERS_KEY_NAME, membersList.toArray()); - groupStructs.add(groupStruct); - } - struct.set(GROUPS_KEY_NAME, groupStructs.toArray()); this.groups = groups; } public DescribeGroupsResponse(Struct struct) { - super(struct); this.groups = new HashMap<>(); for (Object groupObj : struct.getArray(GROUPS_KEY_NAME)) { Struct groupStruct = (Struct) groupObj; @@ -209,10 +180,6 @@ public ByteBuffer memberAssignment() { } } - public static DescribeGroupsResponse parse(ByteBuffer buffer) { - return new DescribeGroupsResponse(CURRENT_SCHEMA.read(buffer)); - } - public static DescribeGroupsResponse fromError(Errors error, List groupIds) { GroupMetadata errorMetadata = GroupMetadata.forError(error); Map groups = new HashMap<>(); @@ -221,4 +188,38 @@ public static DescribeGroupsResponse fromError(Errors error, List groupI return new DescribeGroupsResponse(groups); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.DESCRIBE_GROUPS.id, version)); + + List groupStructs = new ArrayList<>(); + for (Map.Entry groupEntry : groups.entrySet()) { + Struct groupStruct = struct.instance(GROUPS_KEY_NAME); + GroupMetadata group = groupEntry.getValue(); + groupStruct.set(GROUP_ID_KEY_NAME, groupEntry.getKey()); + groupStruct.set(ERROR_CODE_KEY_NAME, group.error.code()); + groupStruct.set(GROUP_STATE_KEY_NAME, group.state); + groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); + groupStruct.set(PROTOCOL_KEY_NAME, group.protocol); + List membersList = new ArrayList<>(); + for (GroupMember member : group.members) { + Struct memberStruct = groupStruct.instance(MEMBERS_KEY_NAME); + memberStruct.set(MEMBER_ID_KEY_NAME, member.memberId); + memberStruct.set(CLIENT_ID_KEY_NAME, member.clientId); + memberStruct.set(CLIENT_HOST_KEY_NAME, member.clientHost); + memberStruct.set(MEMBER_METADATA_KEY_NAME, member.memberMetadata); + memberStruct.set(MEMBER_ASSIGNMENT_KEY_NAME, member.memberAssignment); + membersList.add(memberStruct); + } + groupStruct.set(MEMBERS_KEY_NAME, membersList.toArray()); + groupStructs.add(groupStruct); + } + struct.set(GROUPS_KEY_NAME, groupStructs.toArray()); + + return struct; + } + + public static DescribeGroupsResponse parse(ByteBuffer buffer, short versionId) { + return new DescribeGroupsResponse(ProtoUtils.parseResponse(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer)); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 4f270e15043cf..55fd286d631f8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -92,27 +92,32 @@ public static List> batchByTopic(LinkedHashMap { - private int replicaId = CONSUMER_REPLICA_ID; - private int maxWait; + private final int maxWait; private final int minBytes; + private final int replicaId; + private final LinkedHashMap fetchData; private int maxBytes = DEFAULT_RESPONSE_MAX_BYTES; - private LinkedHashMap fetchData; - public Builder(int maxWait, int minBytes, LinkedHashMap fetchData) { - super(ApiKeys.FETCH); - this.maxWait = maxWait; - this.minBytes = minBytes; - this.fetchData = fetchData; + public static Builder forConsumer(int maxWait, int minBytes, LinkedHashMap fetchData) { + return new Builder(null, CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData); } - public Builder setReplicaId(int replicaId) { - this.replicaId = replicaId; - return this; + public static Builder forReplica(short desiredVersion, int replicaId, int maxWait, int minBytes, + LinkedHashMap fetchData) { + return new Builder(desiredVersion, replicaId, maxWait, minBytes, fetchData); } - public Builder setMaxWait(int maxWait) { + private Builder(Short desiredVersion, int replicaId, int maxWait, int minBytes, + LinkedHashMap fetchData) { + super(ApiKeys.FETCH, desiredVersion); + this.replicaId = replicaId; this.maxWait = maxWait; - return this; + this.minBytes = minBytes; + this.fetchData = fetchData; + } + + public LinkedHashMap fetchData() { + return this.fetchData; } public Builder setMaxBytes(int maxBytes) { @@ -120,19 +125,13 @@ public Builder setMaxBytes(int maxBytes) { return this; } - public LinkedHashMap fetchData() { - return this.fetchData; - } - @Override - public FetchRequest build() { - short version = version(); + public FetchRequest build(short version) { if (version < 3) { maxBytes = -1; } - return new FetchRequest(version, replicaId, maxWait, minBytes, - maxBytes, fetchData); + return new FetchRequest(version, replicaId, maxWait, minBytes, maxBytes, fetchData); } @Override @@ -151,31 +150,7 @@ public String toString() { private FetchRequest(short version, int replicaId, int maxWait, int minBytes, int maxBytes, LinkedHashMap fetchData) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version)), version); - List> topicsData = TopicAndPartitionData.batchByTopic(fetchData); - - struct.set(REPLICA_ID_KEY_NAME, replicaId); - struct.set(MAX_WAIT_KEY_NAME, maxWait); - struct.set(MIN_BYTES_KEY_NAME, minBytes); - if (version >= 3) - struct.set(MAX_BYTES_KEY_NAME, maxBytes); - List topicArray = new ArrayList<>(); - for (TopicAndPartitionData topicEntry : topicsData) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.topic); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.partitions.entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + super(version); this.replicaId = replicaId; this.maxWait = maxWait; this.minBytes = minBytes; @@ -183,8 +158,8 @@ private FetchRequest(short version, int replicaId, int maxWait, int minBytes, in this.fetchData = fetchData; } - public FetchRequest(Struct struct, short versionId) { - super(struct, versionId); + public FetchRequest(Struct struct, short version) { + super(version); replicaId = struct.getInt(REPLICA_ID_KEY_NAME); maxWait = struct.getInt(MAX_WAIT_KEY_NAME); minBytes = struct.getInt(MIN_BYTES_KEY_NAME); @@ -217,8 +192,7 @@ public AbstractResponse getErrorResponse(Throwable e) { responseData.put(entry.getKey(), partitionResponse); } - short versionId = version(); - return new FetchResponse(versionId, responseData, 0); + return new FetchResponse(responseData, 0); } public int replicaId() { @@ -245,11 +219,38 @@ public boolean isFromFollower() { return replicaId >= 0; } - public static FetchRequest parse(ByteBuffer buffer, int versionId) { - return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer), (short) versionId); + public static FetchRequest parse(ByteBuffer buffer, short versionId) { + return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer), versionId); } - public static FetchRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.FETCH.id)); + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version)); + List> topicsData = TopicAndPartitionData.batchByTopic(fetchData); + + struct.set(REPLICA_ID_KEY_NAME, replicaId); + struct.set(MAX_WAIT_KEY_NAME, maxWait); + struct.set(MIN_BYTES_KEY_NAME, minBytes); + if (version >= 3) + struct.set(MAX_BYTES_KEY_NAME, maxBytes); + List topicArray = new ArrayList<>(); + for (TopicAndPartitionData topicEntry : topicsData) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.topic); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.partitions.entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset); + partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 64bd3d3aa360b..dee6a5f53b496 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -23,7 +23,7 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; + import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; import org.apache.kafka.common.record.Records; @@ -39,7 +39,6 @@ */ public class FetchResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names @@ -71,7 +70,7 @@ public class FetchResponse extends AbstractResponse { public static final long INVALID_HIGHWATERMARK = -1L; private final LinkedHashMap responseData; - private final int throttleTime; + private final int throttleTimeMs; public static final class PartitionData { public final Errors error; @@ -91,18 +90,6 @@ public String toString() { } } - /** - * Constructor for version 3. - * - * The entries in `responseData` should be in the same order as the entries in `FetchRequest.fetchData`. - * - * @param responseData fetched data grouped by topic-partition - * @param throttleTime Time in milliseconds the response was throttled - */ - public FetchResponse(LinkedHashMap responseData, int throttleTime) { - this(3, responseData, throttleTime); - } - /** * Constructor for all versions. * @@ -110,17 +97,14 @@ public FetchResponse(LinkedHashMap responseData, * `FetchRequest.fetchData`. * * @param responseData fetched data grouped by topic-partition - * @param throttleTime Time in milliseconds the response was throttled + * @param throttleTimeMs Time in milliseconds the response was throttled */ - public FetchResponse(int version, LinkedHashMap responseData, int throttleTime) { - super(writeStruct(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version)), version, responseData, - throttleTime)); + public FetchResponse(LinkedHashMap responseData, int throttleTimeMs) { this.responseData = responseData; - this.throttleTime = throttleTime; + this.throttleTimeMs = throttleTimeMs; } public FetchResponse(Struct struct) { - super(struct); LinkedHashMap responseData = new LinkedHashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; @@ -137,22 +121,31 @@ public FetchResponse(Struct struct) { } } this.responseData = responseData; - this.throttleTime = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME; + this.throttleTimeMs = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME; + } + + @Override + public Struct toStruct(short version) { + return toStruct(version, responseData, throttleTimeMs); } @Override public Send toSend(String dest, RequestHeader requestHeader) { - ResponseHeader responseHeader = new ResponseHeader(requestHeader.correlationId()); + return toSend(toStruct(requestHeader.apiVersion()), throttleTimeMs, dest, requestHeader); + } + + public Send toSend(Struct responseStruct, int throttleTimeMs, String dest, RequestHeader requestHeader) { + Struct responseHeader = new ResponseHeader(requestHeader.correlationId()).toStruct(); // write the total size and the response header ByteBuffer buffer = ByteBuffer.allocate(responseHeader.sizeOf() + 4); - buffer.putInt(responseHeader.sizeOf() + struct.sizeOf()); + buffer.putInt(responseHeader.sizeOf() + responseStruct.sizeOf()); responseHeader.writeTo(buffer); buffer.rewind(); List sends = new ArrayList<>(); sends.add(new ByteBufferSend(dest, buffer)); - addResponseData(dest, sends); + addResponseData(responseStruct, throttleTimeMs, dest, sends); return new MultiSend(dest, sends); } @@ -160,25 +153,20 @@ public LinkedHashMap responseData() { return responseData; } - public int getThrottleTime() { - return this.throttleTime; + public int throttleTimeMs() { + return this.throttleTimeMs; } - public static FetchResponse parse(ByteBuffer buffer) { - return new FetchResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static FetchResponse parse(ByteBuffer buffer, int version) { + public static FetchResponse parse(ByteBuffer buffer, short version) { return new FetchResponse(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version).read(buffer)); } - private void addResponseData(String dest, List sends) { + private static void addResponseData(Struct struct, int throttleTimeMs, String dest, List sends) { Object[] allTopicData = struct.getArray(RESPONSES_KEY_NAME); if (struct.hasField(THROTTLE_TIME_KEY_NAME)) { - int throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); ByteBuffer buffer = ByteBuffer.allocate(8); - buffer.putInt(throttleTime); + buffer.putInt(throttleTimeMs); buffer.putInt(allTopicData.length); buffer.rewind(); sends.add(new ByteBufferSend(dest, buffer)); @@ -193,7 +181,7 @@ private void addResponseData(String dest, List sends) { addTopicData(dest, sends, (Struct) topicData); } - private void addTopicData(String dest, List sends, Struct topicData) { + private static void addTopicData(String dest, List sends, Struct topicData) { String topic = topicData.getString(TOPIC_KEY_NAME); Object[] allPartitionData = topicData.getArray(PARTITIONS_KEY_NAME); @@ -208,7 +196,7 @@ private void addTopicData(String dest, List sends, Struct topicData) { addPartitionData(dest, sends, (Struct) partitionData); } - private void addPartitionData(String dest, List sends, Struct partitionData) { + private static void addPartitionData(String dest, List sends, Struct partitionData) { Struct header = partitionData.getStruct(PARTITION_HEADER_KEY_NAME); Records records = partitionData.getRecords(RECORD_SET_KEY_NAME); @@ -223,10 +211,8 @@ private void addPartitionData(String dest, List sends, Struct partitionDat sends.add(new RecordsSend(dest, records)); } - private static Struct writeStruct(Struct struct, - int version, - LinkedHashMap responseData, - int throttleTime) { + private static Struct toStruct(short version, LinkedHashMap responseData, int throttleTime) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version)); List> topicsData = FetchRequest.TopicAndPartitionData.batchByTopic(responseData); List topicArray = new ArrayList<>(); for (FetchRequest.TopicAndPartitionData topicEntry: topicsData) { @@ -255,10 +241,8 @@ private static Struct writeStruct(Struct struct, return struct; } - public static int sizeOf(int version, LinkedHashMap responseData) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version)); - writeStruct(struct, version, responseData, 0); - return 4 + struct.sizeOf(); + public static int sizeOf(short version, LinkedHashMap responseData) { + return 4 + toStruct(version, responseData, 0).sizeOf(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java index ed56f3997f7d5..83d6cba27a36d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java @@ -32,8 +32,7 @@ public Builder(String groupId) { } @Override - public GroupCoordinatorRequest build() { - short version = version(); + public GroupCoordinatorRequest build(short version) { return new GroupCoordinatorRequest(this.groupId, version); } @@ -49,14 +48,12 @@ public String toString() { private final String groupId; private GroupCoordinatorRequest(String groupId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.GROUP_COORDINATOR.id, version)), - version); - struct.set(GROUP_ID_KEY_NAME, groupId); + super(version); this.groupId = groupId; } public GroupCoordinatorRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); groupId = struct.getString(GROUP_ID_KEY_NAME); } @@ -76,12 +73,15 @@ public String groupId() { return groupId; } - public static GroupCoordinatorRequest parse(ByteBuffer buffer, int versionId) { + public static GroupCoordinatorRequest parse(ByteBuffer buffer, short versionId) { return new GroupCoordinatorRequest(ProtoUtils.parseRequest(ApiKeys.GROUP_COORDINATOR.id, versionId, buffer), - (short) versionId); + versionId); } - public static GroupCoordinatorRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.GROUP_COORDINATOR.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.GROUP_COORDINATOR.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java index fc3d358668cac..c13cf3c3ab707 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java @@ -16,14 +16,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class GroupCoordinatorResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.GROUP_COORDINATOR.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String COORDINATOR_KEY_NAME = "coordinator"; @@ -45,19 +43,11 @@ public class GroupCoordinatorResponse extends AbstractResponse { private final Node node; public GroupCoordinatorResponse(Errors error, Node node) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); - coordinator.set(NODE_ID_KEY_NAME, node.id()); - coordinator.set(HOST_KEY_NAME, node.host()); - coordinator.set(PORT_KEY_NAME, node.port()); - struct.set(COORDINATOR_KEY_NAME, coordinator); this.error = error; this.node = node; } public GroupCoordinatorResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME); int nodeId = broker.getInt(NODE_ID_KEY_NAME); @@ -74,7 +64,19 @@ public Node node() { return node; } - public static GroupCoordinatorResponse parse(ByteBuffer buffer) { - return new GroupCoordinatorResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.GROUP_COORDINATOR.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); + coordinator.set(NODE_ID_KEY_NAME, node.id()); + coordinator.set(HOST_KEY_NAME, node.host()); + coordinator.set(PORT_KEY_NAME, node.port()); + struct.set(COORDINATOR_KEY_NAME, coordinator); + return struct; + } + + public static GroupCoordinatorResponse parse(ByteBuffer buffer, short version) { + return new GroupCoordinatorResponse(ProtoUtils.parseResponse(ApiKeys.GROUP_COORDINATOR.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 7e79c8a4ff212..444083075c51c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -37,8 +37,8 @@ public Builder(String groupId, int groupGenerationId, String memberId) { } @Override - public HeartbeatRequest build() { - return new HeartbeatRequest(groupId, groupGenerationId, memberId, version()); + public HeartbeatRequest build(short version) { + return new HeartbeatRequest(groupId, groupGenerationId, memberId, version); } @Override @@ -58,18 +58,14 @@ public String toString() { private final String memberId; private HeartbeatRequest(String groupId, int groupGenerationId, String memberId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.HEARTBEAT.id, version)), - version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); + super(version); this.groupId = groupId; this.groupGenerationId = groupGenerationId; this.memberId = memberId; } - public HeartbeatRequest(Struct struct, short versionId) { - super(struct, versionId); + public HeartbeatRequest(Struct struct, short version) { + super(version); groupId = struct.getString(GROUP_ID_KEY_NAME); groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME); memberId = struct.getString(MEMBER_ID_KEY_NAME); @@ -99,11 +95,16 @@ public String memberId() { return memberId; } - public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) { - return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer), (short) versionId); + public static HeartbeatRequest parse(ByteBuffer buffer, short versionId) { + return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer), versionId); } - public static HeartbeatRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.HEARTBEAT.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); + struct.set(MEMBER_ID_KEY_NAME, memberId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index f36dec446f559..4cca846b3f6bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -15,14 +15,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class HeartbeatResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -35,17 +33,13 @@ public class HeartbeatResponse extends AbstractResponse { * REBALANCE_IN_PROGRESS (27) * GROUP_AUTHORIZATION_FAILED (30) */ - private final Errors error; public HeartbeatResponse(Errors error) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); this.error = error; } public HeartbeatResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); } @@ -53,7 +47,14 @@ public Errors error() { return error; } - public static HeartbeatResponse parse(ByteBuffer buffer) { - return new HeartbeatResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.HEARTBEAT.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + return struct; + } + + public static HeartbeatResponse parse(ByteBuffer buffer, short version) { + return new HeartbeatResponse(ProtoUtils.parseResponse(ApiKeys.HEARTBEAT.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index ad0cdd0f7fb8d..37906a626c74e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -85,8 +85,7 @@ public Builder setRebalanceTimeout(int rebalanceTimeout) { } @Override - public JoinGroupRequest build() { - short version = version(); + public JoinGroupRequest build(short version) { if (version < 1) { rebalanceTimeout = -1; } @@ -112,23 +111,7 @@ public String toString() { private JoinGroupRequest(short version, String groupId, int sessionTimeout, int rebalanceTimeout, String memberId, String protocolType, List groupProtocols) { - super(new Struct(ProtoUtils. - requestSchema(ApiKeys.JOIN_GROUP.id, version)), version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); - if (version >= 1) { - struct.set(REBALANCE_TIMEOUT_KEY_NAME, rebalanceTimeout); - } - struct.set(MEMBER_ID_KEY_NAME, memberId); - struct.set(PROTOCOL_TYPE_KEY_NAME, protocolType); - List groupProtocolsList = new ArrayList<>(groupProtocols.size()); - for (ProtocolMetadata protocol : groupProtocols) { - Struct protocolStruct = struct.instance(GROUP_PROTOCOLS_KEY_NAME); - protocolStruct.set(PROTOCOL_NAME_KEY_NAME, protocol.name); - protocolStruct.set(PROTOCOL_METADATA_KEY_NAME, protocol.metadata); - groupProtocolsList.add(protocolStruct); - } - struct.set(GROUP_PROTOCOLS_KEY_NAME, groupProtocolsList.toArray()); + super(version); this.groupId = groupId; this.sessionTimeout = sessionTimeout; this.rebalanceTimeout = rebalanceTimeout; @@ -138,7 +121,7 @@ private JoinGroupRequest(short version, String groupId, int sessionTimeout, } public JoinGroupRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); groupId = struct.getString(GROUP_ID_KEY_NAME); sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME); @@ -169,7 +152,6 @@ public AbstractResponse getErrorResponse(Throwable e) { case 0: case 1: return new JoinGroupResponse( - versionId, Errors.forException(e), JoinGroupResponse.UNKNOWN_GENERATION_ID, JoinGroupResponse.UNKNOWN_PROTOCOL, @@ -207,12 +189,29 @@ public String protocolType() { return protocolType; } - public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) { - return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, versionId, buffer), - (short) versionId); + public static JoinGroupRequest parse(ByteBuffer buffer, short version) { + return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, version, buffer), version); } - public static JoinGroupRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id)); + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.JOIN_GROUP.id, version)); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); + if (version >= 1) { + struct.set(REBALANCE_TIMEOUT_KEY_NAME, rebalanceTimeout); + } + struct.set(MEMBER_ID_KEY_NAME, memberId); + struct.set(PROTOCOL_TYPE_KEY_NAME, protocolType); + List groupProtocolsList = new ArrayList<>(groupProtocols.size()); + for (ProtocolMetadata protocol : groupProtocols) { + Struct protocolStruct = struct.instance(GROUP_PROTOCOLS_KEY_NAME); + protocolStruct.set(PROTOCOL_NAME_KEY_NAME, protocol.name); + protocolStruct.set(PROTOCOL_METADATA_KEY_NAME, protocol.metadata); + groupProtocolsList.add(protocolStruct); + } + struct.set(GROUP_PROTOCOLS_KEY_NAME, groupProtocolsList.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index bc9366a91cc11..d2a323b157bf7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -26,8 +25,6 @@ public class JoinGroupResponse extends AbstractResponse { - private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id); - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -67,33 +64,6 @@ public JoinGroupResponse(Errors error, String memberId, String leaderId, Map groupMembers) { - this(CURRENT_VERSION, error, generationId, groupProtocol, memberId, leaderId, groupMembers); - } - - public JoinGroupResponse(int version, - Errors error, - int generationId, - String groupProtocol, - String memberId, - String leaderId, - Map groupMembers) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version))); - - struct.set(ERROR_CODE_KEY_NAME, error.code()); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(GROUP_PROTOCOL_KEY_NAME, groupProtocol); - struct.set(MEMBER_ID_KEY_NAME, memberId); - struct.set(LEADER_ID_KEY_NAME, leaderId); - - List memberArray = new ArrayList<>(); - for (Map.Entry entries: groupMembers.entrySet()) { - Struct memberData = struct.instance(MEMBERS_KEY_NAME); - memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); - memberData.set(MEMBER_METADATA_KEY_NAME, entries.getValue()); - memberArray.add(memberData); - } - struct.set(MEMBERS_KEY_NAME, memberArray.toArray()); - this.error = error; this.generationId = generationId; this.groupProtocol = groupProtocol; @@ -103,7 +73,6 @@ public JoinGroupResponse(int version, } public JoinGroupResponse(Struct struct) { - super(struct); members = new HashMap<>(); for (Object memberDataObj : struct.getArray(MEMBERS_KEY_NAME)) { @@ -147,11 +116,29 @@ public Map members() { return members; } - public static JoinGroupResponse parse(ByteBuffer buffer, int version) { - return new JoinGroupResponse(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version).read(buffer)); + public static JoinGroupResponse parse(ByteBuffer buffer, short version) { + return new JoinGroupResponse(ProtoUtils.parseResponse(ApiKeys.JOIN_GROUP.id, version, buffer)); } - public static JoinGroupResponse parse(ByteBuffer buffer) { - return new JoinGroupResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version)); + + struct.set(ERROR_CODE_KEY_NAME, error.code()); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(GROUP_PROTOCOL_KEY_NAME, groupProtocol); + struct.set(MEMBER_ID_KEY_NAME, memberId); + struct.set(LEADER_ID_KEY_NAME, leaderId); + + List memberArray = new ArrayList<>(); + for (Map.Entry entries : members.entrySet()) { + Struct memberData = struct.instance(MEMBERS_KEY_NAME); + memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); + memberData.set(MEMBER_METADATA_KEY_NAME, entries.getValue()); + memberArray.add(memberData); + } + struct.set(MEMBERS_KEY_NAME, memberArray.toArray()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index fde184aee18be..c564b437d9d76 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -69,9 +69,8 @@ public Builder(int controllerId, int controllerEpoch, } @Override - public LeaderAndIsrRequest build() { - return new LeaderAndIsrRequest(controllerId, controllerEpoch, partitionStates, - liveLeaders, version()); + public LeaderAndIsrRequest build(short version) { + return new LeaderAndIsrRequest(controllerId, controllerEpoch, partitionStates, liveLeaders, version); } @Override @@ -94,46 +93,15 @@ public String toString() { private LeaderAndIsrRequest(int controllerId, int controllerEpoch, Map partitionStates, Set liveLeaders, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LEADER_AND_ISR.id, version)), - version); - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); - - List partitionStatesData = new ArrayList<>(partitionStates.size()); - for (Map.Entry entry : partitionStates.entrySet()) { - Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); - TopicPartition topicPartition = entry.getKey(); - partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); - partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); - PartitionState partitionState = entry.getValue(); - partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); - partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); - partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); - partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); - partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); - partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); - partitionStatesData.add(partitionStateData); - } - struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); - - List leadersData = new ArrayList<>(liveLeaders.size()); - for (Node leader : liveLeaders) { - Struct leaderData = struct.instance(LIVE_LEADERS_KEY_NAME); - leaderData.set(END_POINT_ID_KEY_NAME, leader.id()); - leaderData.set(HOST_KEY_NAME, leader.host()); - leaderData.set(PORT_KEY_NAME, leader.port()); - leadersData.add(leaderData); - } - struct.set(LIVE_LEADERS_KEY_NAME, leadersData.toArray()); - + super(version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.partitionStates = partitionStates; this.liveLeaders = liveLeaders; } - public LeaderAndIsrRequest(Struct struct, short versionId) { - super(struct, versionId); + public LeaderAndIsrRequest(Struct struct, short version) { + super(version); Map partitionStates = new HashMap<>(); for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) { @@ -176,6 +144,42 @@ public LeaderAndIsrRequest(Struct struct, short versionId) { this.liveLeaders = leaders; } + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LEADER_AND_ISR.id, version)); + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + + List partitionStatesData = new ArrayList<>(partitionStates.size()); + for (Map.Entry entry : partitionStates.entrySet()) { + Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); + TopicPartition topicPartition = entry.getKey(); + partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); + partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); + PartitionState partitionState = entry.getValue(); + partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); + partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); + partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); + partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); + partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); + partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + partitionStatesData.add(partitionStateData); + } + struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); + + List leadersData = new ArrayList<>(liveLeaders.size()); + for (Node leader : liveLeaders) { + Struct leaderData = struct.instance(LIVE_LEADERS_KEY_NAME); + leaderData.set(END_POINT_ID_KEY_NAME, leader.id()); + leaderData.set(HOST_KEY_NAME, leader.host()); + leaderData.set(PORT_KEY_NAME, leader.port()); + leadersData.add(leaderData); + } + struct.set(LIVE_LEADERS_KEY_NAME, leadersData.toArray()); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { Map responses = new HashMap<>(partitionStates.size()); @@ -209,12 +213,8 @@ public Set liveLeaders() { return liveLeaders; } - public static LeaderAndIsrRequest parse(ByteBuffer buffer, int versionId) { - return new LeaderAndIsrRequest(ProtoUtils.parseRequest(ApiKeys.LEADER_AND_ISR.id, versionId, buffer), - (short) versionId); + public static LeaderAndIsrRequest parse(ByteBuffer buffer, short versionId) { + return new LeaderAndIsrRequest(ProtoUtils.parseRequest(ApiKeys.LEADER_AND_ISR.id, versionId, buffer), versionId); } - public static LeaderAndIsrRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LEADER_AND_ISR.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java index 4d0a05dfad6cd..2b02daf2fd277 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -17,7 +17,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,7 +26,6 @@ import java.util.Map; public class LeaderAndIsrResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LEADER_AND_ISR.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String PARTITIONS_KEY_NAME = "partitions"; @@ -45,33 +43,12 @@ public class LeaderAndIsrResponse extends AbstractResponse { private final Map responses; - public LeaderAndIsrResponse(Map responses) { - this(Errors.NONE, responses); - } - public LeaderAndIsrResponse(Errors error, Map responses) { - super(new Struct(CURRENT_SCHEMA)); - - List responseDatas = new ArrayList<>(responses.size()); - for (Map.Entry response : responses.entrySet()) { - Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); - TopicPartition partition = response.getKey(); - partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); - partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); - partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); - responseDatas.add(partitionData); - } - - struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - this.responses = responses; this.error = error; } public LeaderAndIsrResponse(Struct struct) { - super(struct); - responses = new HashMap<>(); for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { Struct responseData = (Struct) responseDataObj; @@ -92,12 +69,27 @@ public Errors error() { return error; } - public static LeaderAndIsrResponse parse(ByteBuffer buffer, int version) { + public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) { return new LeaderAndIsrResponse(ProtoUtils.parseResponse(ApiKeys.LEADER_AND_ISR.id, version, buffer)); } - public static LeaderAndIsrResponse parse(ByteBuffer buffer) { - return new LeaderAndIsrResponse(CURRENT_SCHEMA.read(buffer)); - } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LEADER_AND_ISR.id, version)); + + List responseDatas = new ArrayList<>(responses.size()); + for (Map.Entry response : responses.entrySet()) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + TopicPartition partition = response.getKey(); + partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); + partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); + responseDatas.add(partitionData); + } + struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java index 2a7b70e4c2be6..16622e41e0336 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java @@ -33,8 +33,8 @@ public Builder(String groupId, String memberId) { } @Override - public LeaveGroupRequest build() { - return new LeaveGroupRequest(groupId, memberId, version()); + public LeaveGroupRequest build(short version) { + return new LeaveGroupRequest(groupId, memberId, version); } @Override @@ -52,16 +52,13 @@ public String toString() { private final String memberId; private LeaveGroupRequest(String groupId, String memberId, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LEAVE_GROUP.id, version)), - version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(MEMBER_ID_KEY_NAME, memberId); + super(version); this.groupId = groupId; this.memberId = memberId; } public LeaveGroupRequest(Struct struct, short version) { - super(struct, version); + super(version); groupId = struct.getString(GROUP_ID_KEY_NAME); memberId = struct.getString(MEMBER_ID_KEY_NAME); } @@ -86,12 +83,15 @@ public String memberId() { return memberId; } - public static LeaveGroupRequest parse(ByteBuffer buffer, int versionId) { - return new LeaveGroupRequest(ProtoUtils.parseRequest(ApiKeys.LEAVE_GROUP.id, versionId, buffer), - (short) versionId); + public static LeaveGroupRequest parse(ByteBuffer buffer, short versionId) { + return new LeaveGroupRequest(ProtoUtils.parseRequest(ApiKeys.LEAVE_GROUP.id, versionId, buffer), versionId); } - public static LeaveGroupRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LEAVE_GROUP.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LEAVE_GROUP.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(MEMBER_ID_KEY_NAME, memberId); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java index bd1c84d49c148..a28816a5b0ecc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java @@ -12,16 +12,15 @@ */ package org.apache.kafka.common.requests; -import java.nio.ByteBuffer; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; +import java.nio.ByteBuffer; + public class LeaveGroupResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LEAVE_GROUP.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -33,16 +32,13 @@ public class LeaveGroupResponse extends AbstractResponse { * UNKNOWN_CONSUMER_ID (25) * GROUP_AUTHORIZATION_FAILED (30) */ - private final Errors error; + public LeaveGroupResponse(Errors error) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); this.error = error; } public LeaveGroupResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); } @@ -50,7 +46,15 @@ public Errors error() { return error; } - public static LeaveGroupResponse parse(ByteBuffer buffer) { - return new LeaveGroupResponse(CURRENT_SCHEMA.read(buffer)); + @Override + public Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LEAVE_GROUP.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + return struct; } + + public static LeaveGroupResponse parse(ByteBuffer buffer, short versionId) { + return new LeaveGroupResponse(ProtoUtils.parseResponse(ApiKeys.LEAVE_GROUP.id, versionId, buffer)); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java index 235f4e4174183..badb527f38a34 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java @@ -27,8 +27,8 @@ public Builder() { } @Override - public ListGroupsRequest build() { - return new ListGroupsRequest(version()); + public ListGroupsRequest build(short version) { + return new ListGroupsRequest(version); } @Override @@ -38,12 +38,11 @@ public String toString() { } public ListGroupsRequest(short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_GROUPS.id, version)), - version); + super(version); } public ListGroupsRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); } @Override @@ -58,12 +57,12 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static ListGroupsRequest parse(ByteBuffer buffer, int versionId) { - return new ListGroupsRequest(ProtoUtils.parseRequest(ApiKeys.LIST_GROUPS.id, versionId, buffer), - (short) versionId); + public static ListGroupsRequest parse(ByteBuffer buffer, short versionId) { + return new ListGroupsRequest(ProtoUtils.parseRequest(ApiKeys.LIST_GROUPS.id, versionId, buffer), versionId); } - public static ListGroupsRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LIST_GROUPS.id)); + @Override + protected Struct toStruct() { + return new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_GROUPS.id, version())); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index f4210640e712a..e05a4b1df5817 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -25,8 +24,6 @@ public class ListGroupsResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_GROUPS.id); - public static final String ERROR_CODE_KEY_NAME = "error_code"; public static final String GROUPS_KEY_NAME = "groups"; public static final String GROUP_ID_KEY_NAME = "group_id"; @@ -43,22 +40,11 @@ public class ListGroupsResponse extends AbstractResponse { private final List groups; public ListGroupsResponse(Errors error, List groups) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - List groupList = new ArrayList<>(); - for (Group group : groups) { - Struct groupStruct = struct.instance(GROUPS_KEY_NAME); - groupStruct.set(GROUP_ID_KEY_NAME, group.groupId); - groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); - groupList.add(groupStruct); - } - struct.set(GROUPS_KEY_NAME, groupList.toArray()); this.error = error; this.groups = groups; } public ListGroupsResponse(Struct struct) { - super(struct); this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); this.groups = new ArrayList<>(); for (Object groupObj : struct.getArray(GROUPS_KEY_NAME)) { @@ -96,12 +82,27 @@ public String protocolType() { } - public static ListGroupsResponse parse(ByteBuffer buffer) { - return new ListGroupsResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_GROUPS.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + List groupList = new ArrayList<>(); + for (Group group : groups) { + Struct groupStruct = struct.instance(GROUPS_KEY_NAME); + groupStruct.set(GROUP_ID_KEY_NAME, group.groupId); + groupStruct.set(PROTOCOL_TYPE_KEY_NAME, group.protocolType); + groupList.add(groupStruct); + } + struct.set(GROUPS_KEY_NAME, groupList.toArray()); + return struct; } public static ListGroupsResponse fromError(Errors error) { return new ListGroupsResponse(error, Collections.emptyList()); } + public static ListGroupsResponse parse(ByteBuffer buffer, short versionId) { + return new ListGroupsResponse(ProtoUtils.parseResponse(ApiKeys.LIST_GROUPS.id, versionId, buffer)); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 6214a5601bdfc..3e2ad7c1571a6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -60,16 +60,21 @@ public class ListOffsetRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { private final int replicaId; + private final short minVersion; private Map offsetData = null; private Map partitionTimestamps = null; - private short minVersion = (short) 0; - public Builder() { - this(CONSUMER_REPLICA_ID); + public static Builder forReplica(short desiredVersion, int replicaId) { + return new Builder((short) 0, desiredVersion, replicaId); } - public Builder(int replicaId) { - super(ApiKeys.LIST_OFFSETS); + public static Builder forConsumer(short minVersion) { + return new Builder(minVersion, null, CONSUMER_REPLICA_ID); + } + + private Builder(short minVersion, Short desiredVersion, int replicaId) { + super(ApiKeys.LIST_OFFSETS, desiredVersion); + this.minVersion = minVersion; this.replicaId = replicaId; } @@ -84,8 +89,7 @@ public Builder setTargetTimes(Map partitionTimestamps) { } @Override - public ListOffsetRequest build() { - short version = version(); + public ListOffsetRequest build(short version) { if (version < minVersion) { throw new UnsupportedVersionException("Cannot create a v" + version + " ListOffsetRequest because " + "we require features supported only in " + minVersion + " or later."); @@ -117,14 +121,6 @@ public ListOffsetRequest build() { return new ListOffsetRequest(replicaId, m, version); } - /** - * Set the minimum version which we will produce for this request. - */ - public Builder setMinVersion(short minVersion) { - this.minVersion = minVersion; - return this; - } - @Override public String toString() { StringBuilder bld = new StringBuilder(); @@ -170,44 +166,15 @@ public String toString() { */ @SuppressWarnings("unchecked") private ListOffsetRequest(int replicaId, Map targetTimes, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_OFFSETS.id, version)), version); - Map> topicsData = - CollectionUtils.groupDataByTopic((Map) targetTimes); - - struct.set(REPLICA_ID_KEY_NAME, replicaId); - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - if (version == 0) { - PartitionData offsetPartitionData = (PartitionData) partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); - partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets); - partitionArray.add(partitionData); - } else { - Long timestamp = (Long) partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(TIMESTAMP_KEY_NAME, timestamp); - partitionArray.add(partitionData); - } - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + super(version); this.replicaId = replicaId; this.offsetData = version == 0 ? (Map) targetTimes : null; this.partitionTimestamps = version == 1 ? (Map) targetTimes : null; this.duplicatePartitions = Collections.emptySet(); } - public ListOffsetRequest(Struct struct, short versionId) { - super(struct, versionId); + public ListOffsetRequest(Struct struct, short version) { + super(version); Set duplicatePartitions = new HashSet<>(); replicaId = struct.getInt(REPLICA_ID_KEY_NAME); offsetData = new HashMap<>(); @@ -236,17 +203,19 @@ public ListOffsetRequest(Struct struct, short versionId) { @Override @SuppressWarnings("deprecation") public AbstractResponse getErrorResponse(Throwable e) { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); short versionId = version(); if (versionId == 0) { for (Map.Entry entry : offsetData.entrySet()) { - ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e), new ArrayList()); + ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData( + Errors.forException(e), Collections.emptyList()); responseData.put(entry.getKey(), partitionResponse); } } else { for (Map.Entry entry : partitionTimestamps.entrySet()) { - ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData(Errors.forException(e), -1L, -1L); + ListOffsetResponse.PartitionData partitionResponse = new ListOffsetResponse.PartitionData( + Errors.forException(e), -1L, -1L); responseData.put(entry.getKey(), partitionResponse); } } @@ -254,7 +223,7 @@ public AbstractResponse getErrorResponse(Throwable e) { switch (versionId) { case 0: case 1: - return new ListOffsetResponse(responseData, versionId); + return new ListOffsetResponse(responseData); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id))); @@ -278,12 +247,44 @@ public Set duplicatePartitions() { return duplicatePartitions; } - public static ListOffsetRequest parse(ByteBuffer buffer, int versionId) { - return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer), - (short) versionId); + public static ListOffsetRequest parse(ByteBuffer buffer, short versionId) { + return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer), versionId); } - public static ListOffsetRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id)); + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_OFFSETS.id, version)); + + Map targetTimes = partitionTimestamps == null ? offsetData : partitionTimestamps; + Map> topicsData = CollectionUtils.groupDataByTopic(targetTimes); + + struct.set(REPLICA_ID_KEY_NAME, replicaId); + List topicArray = new ArrayList<>(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + if (version == 0) { + PartitionData offsetPartitionData = (PartitionData) partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); + partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets); + partitionArray.add(partitionData); + } else { + Long timestamp = (Long) partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(TIMESTAMP_KEY_NAME, timestamp); + partitionArray.add(partitionData); + } + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index b815a53d6f9c9..cb3bafcc4eb80 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.Utils; @@ -35,7 +34,6 @@ public class ListOffsetResponse extends AbstractResponse { public static final long UNKNOWN_TIMESTAMP = -1L; public static final long UNKNOWN_OFFSET = -1L; - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names @@ -61,8 +59,6 @@ public class ListOffsetResponse extends AbstractResponse { private static final String TIMESTAMP_KEY_NAME = "timestamp"; private static final String OFFSET_KEY_NAME = "offset"; - private final Map responseData; - public static final class PartitionData { public final Errors error; // The offsets list is only used in ListOffsetResponse v0. @@ -110,46 +106,17 @@ public String toString() { } } + private final Map responseData; + /** - * Constructor for ListOffsetResponse v0. + * Constructor for all versions. */ - @Deprecated public ListOffsetResponse(Map responseData) { - this(responseData, 0); - } - - public ListOffsetResponse(Map responseData, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_OFFSETS.id, version))); - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData offsetPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.error.code()); - if (version == 0) - partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray()); - else { - partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); - partitionData.set(OFFSET_KEY_NAME, offsetPartitionData.offset); - } - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); this.responseData = responseData; } public ListOffsetResponse(Struct struct) { - super(struct); - responseData = new HashMap(); + responseData = new HashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; String topic = topicResponse.getString(TOPIC_KEY_NAME); @@ -178,11 +145,38 @@ public Map responseData() { return responseData; } - public static ListOffsetResponse parse(ByteBuffer buffer) { - return new ListOffsetResponse(CURRENT_SCHEMA.read(buffer)); + public static ListOffsetResponse parse(ByteBuffer buffer, short version) { + return new ListOffsetResponse(ProtoUtils.parseResponse(ApiKeys.LIST_OFFSETS.id, version, buffer)); } - public static ListOffsetResponse parse(ByteBuffer buffer, int version) { - return new ListOffsetResponse(ProtoUtils.responseSchema(ApiKeys.LIST_OFFSETS.id, version).read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_OFFSETS.id, version)); + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + + List topicArray = new ArrayList<>(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData offsetPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.error.code()); + if (version == 0) + partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray()); + else { + partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); + partitionData.set(OFFSET_KEY_NAME, offsetPartitionData.offset); + } + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index 16af1b74f6d39..f31315fbc43cc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -51,8 +51,7 @@ public boolean isAllTopics() { } @Override - public MetadataRequest build() { - short version = version(); + public MetadataRequest build(short version) { if (version < 1) { throw new UnsupportedVersionException("MetadataRequest " + "versions older than 1 are not supported."); @@ -79,27 +78,18 @@ public String toString() { private final List topics; - public static MetadataRequest allTopics(short version) { - return new MetadataRequest.Builder(null).setVersion(version).build(); - } - /** * In v0 null is not allowed and and empty list indicates requesting all topics. * Note: modern clients do not support sending v0 requests. * In v1 null indicates requesting all topics, and an empty list indicates requesting no topics. */ public MetadataRequest(List topics, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id, version)), - version); - if (topics == null) - struct.set(TOPICS_KEY_NAME, null); - else - struct.set(TOPICS_KEY_NAME, topics.toArray()); + super(version); this.topics = topics; } public MetadataRequest(Struct struct, short version) { - super(struct, version); + super(version); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); if (topicArray != null) { topics = new ArrayList<>(); @@ -127,7 +117,7 @@ public AbstractResponse getErrorResponse(Throwable e) { case 0: case 1: case 2: - return new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas, versionId); + return new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); @@ -142,12 +132,17 @@ public List topics() { return topics; } - public static MetadataRequest parse(ByteBuffer buffer, int versionId) { - return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer), - (short) versionId); + public static MetadataRequest parse(ByteBuffer buffer, short versionId) { + return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer), versionId); } - public static MetadataRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.METADATA.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id, version())); + if (topics == null) + struct.set(TOPICS_KEY_NAME, null); + else + struct.set(TOPICS_KEY_NAME, topics.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index a8baee52f3a08..268bf8438a12c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -31,7 +31,6 @@ public class MetadataResponse extends AbstractResponse { - private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.METADATA.id); private static final String BROKERS_KEY_NAME = "brokers"; private static final String TOPIC_METADATA_KEY_NAME = "topic_metadata"; @@ -83,78 +82,16 @@ public class MetadataResponse extends AbstractResponse { private final String clusterId; /** - * Constructor for the latest version + * Constructor for all versions. */ public MetadataResponse(List brokers, String clusterId, int controllerId, List topicMetadata) { - this(brokers, clusterId, controllerId, topicMetadata, CURRENT_VERSION); - } - - /** - * Constructor for a specific version - */ - public MetadataResponse(List brokers, String clusterId, int controllerId, List topicMetadata, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version))); this.brokers = brokers; this.controller = getControllerNode(controllerId, brokers); this.topicMetadata = topicMetadata; this.clusterId = clusterId; - - List brokerArray = new ArrayList<>(); - for (Node node : brokers) { - Struct broker = struct.instance(BROKERS_KEY_NAME); - broker.set(NODE_ID_KEY_NAME, node.id()); - broker.set(HOST_KEY_NAME, node.host()); - broker.set(PORT_KEY_NAME, node.port()); - // This field only exists in v1+ - if (broker.hasField(RACK_KEY_NAME)) - broker.set(RACK_KEY_NAME, node.rack()); - brokerArray.add(broker); - } - struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); - - // This field only exists in v1+ - if (struct.hasField(CONTROLLER_ID_KEY_NAME)) - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - - // This field only exists in v2+ - if (struct.hasField(CLUSTER_ID_KEY_NAME)) - struct.set(CLUSTER_ID_KEY_NAME, clusterId); - - List topicMetadataArray = new ArrayList<>(topicMetadata.size()); - for (TopicMetadata metadata : topicMetadata) { - Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, metadata.topic); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, metadata.error.code()); - // This field only exists in v1+ - if (topicData.hasField(IS_INTERNAL_KEY_NAME)) - topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal()); - - List partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size()); - for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) { - Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); - partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, partitionMetadata.error.code()); - partitionData.set(PARTITION_KEY_NAME, partitionMetadata.partition); - partitionData.set(LEADER_KEY_NAME, partitionMetadata.leader.id()); - ArrayList replicas = new ArrayList<>(partitionMetadata.replicas.size()); - for (Node node : partitionMetadata.replicas) - replicas.add(node.id()); - partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); - ArrayList isr = new ArrayList<>(partitionMetadata.isr.size()); - for (Node node : partitionMetadata.isr) - isr.add(node.id()); - partitionData.set(ISR_KEY_NAME, isr.toArray()); - partitionMetadataArray.add(partitionData); - - } - topicData.set(PARTITION_METADATA_KEY_NAME, partitionMetadataArray.toArray()); - topicMetadataArray.add(topicData); - } - struct.set(TOPIC_METADATA_KEY_NAME, topicMetadataArray.toArray()); } public MetadataResponse(Struct struct) { - super(struct); - Map brokers = new HashMap<>(); Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME); for (int i = 0; i < brokerStructs.length; i++) { @@ -317,12 +254,8 @@ public String clusterId() { return this.clusterId; } - public static MetadataResponse parse(ByteBuffer buffer) { - return parse(buffer, CURRENT_VERSION); - } - - public static MetadataResponse parse(ByteBuffer buffer, int version) { - return new MetadataResponse(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version).read(buffer)); + public static MetadataResponse parse(ByteBuffer buffer, short version) { + return new MetadataResponse(ProtoUtils.parseResponse(ApiKeys.METADATA.id, version, buffer)); } public static class TopicMetadata { @@ -400,4 +333,60 @@ public List isr() { } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version)); + List brokerArray = new ArrayList<>(); + for (Node node : brokers) { + Struct broker = struct.instance(BROKERS_KEY_NAME); + broker.set(NODE_ID_KEY_NAME, node.id()); + broker.set(HOST_KEY_NAME, node.host()); + broker.set(PORT_KEY_NAME, node.port()); + // This field only exists in v1+ + if (broker.hasField(RACK_KEY_NAME)) + broker.set(RACK_KEY_NAME, node.rack()); + brokerArray.add(broker); + } + struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); + + // This field only exists in v1+ + if (struct.hasField(CONTROLLER_ID_KEY_NAME)) + struct.set(CONTROLLER_ID_KEY_NAME, controller == null ? NO_CONTROLLER_ID : controller.id()); + + // This field only exists in v2+ + if (struct.hasField(CLUSTER_ID_KEY_NAME)) + struct.set(CLUSTER_ID_KEY_NAME, clusterId); + + List topicMetadataArray = new ArrayList<>(topicMetadata.size()); + for (TopicMetadata metadata : topicMetadata) { + Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, metadata.topic); + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, metadata.error.code()); + // This field only exists in v1+ + if (topicData.hasField(IS_INTERNAL_KEY_NAME)) + topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal()); + + List partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size()); + for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) { + Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); + partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, partitionMetadata.error.code()); + partitionData.set(PARTITION_KEY_NAME, partitionMetadata.partition); + partitionData.set(LEADER_KEY_NAME, partitionMetadata.leader.id()); + ArrayList replicas = new ArrayList<>(partitionMetadata.replicas.size()); + for (Node node : partitionMetadata.replicas) + replicas.add(node.id()); + partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); + ArrayList isr = new ArrayList<>(partitionMetadata.isr.size()); + for (Node node : partitionMetadata.isr) + isr.add(node.id()); + partitionData.set(ISR_KEY_NAME, isr.toArray()); + partitionMetadataArray.add(partitionData); + + } + topicData.set(PARTITION_METADATA_KEY_NAME, partitionMetadataArray.toArray()); + topicMetadataArray.add(topicData); + } + struct.set(TOPIC_METADATA_KEY_NAME, topicMetadataArray.toArray()); + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 6dd1197422214..bf14f101a71b8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -124,17 +124,15 @@ public Builder setRetentionTime(long retentionTime) { } @Override - public OffsetCommitRequest build() { - short version = version(); + public OffsetCommitRequest build(short version) { switch (version) { case 0: - return new OffsetCommitRequest(groupId, offsetData); + return new OffsetCommitRequest(groupId, DEFAULT_GENERATION_ID, DEFAULT_MEMBER_ID, + DEFAULT_RETENTION_TIME, offsetData, version); case 1: - return new OffsetCommitRequest(groupId, generationId, memberId, - offsetData); case 2: - return new OffsetCommitRequest(groupId, generationId, memberId, - retentionTime, offsetData, version); + long retentionTime = version == 1 ? DEFAULT_RETENTION_TIME : this.retentionTime; + return new OffsetCommitRequest(groupId, generationId, memberId, retentionTime, offsetData, version); default: throw new UnsupportedVersionException("Unsupported version " + version); } @@ -154,57 +152,9 @@ public String toString() { } } - /** - * Constructor for version 0. - * @param groupId - * @param offsetData - */ - private OffsetCommitRequest(String groupId, Map offsetData) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)), (short) 0); - - initCommonFields(groupId, offsetData); - this.groupId = groupId; - this.generationId = DEFAULT_GENERATION_ID; - this.memberId = DEFAULT_MEMBER_ID; - this.retentionTime = DEFAULT_RETENTION_TIME; - this.offsetData = offsetData; - } - - /** - * Constructor for version 1. - * @param groupId - * @param generationId - * @param memberId - * @param offsetData - */ - private OffsetCommitRequest(String groupId, int generationId, String memberId, Map offsetData) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1)), (short) 1); - - initCommonFields(groupId, offsetData); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); - this.groupId = groupId; - this.generationId = generationId; - this.memberId = memberId; - this.retentionTime = DEFAULT_RETENTION_TIME; - this.offsetData = offsetData; - } - - /** - * Constructor for version 2 and above. - * @param groupId - * @param generationId - * @param memberId - * @param retentionTime - * @param offsetData - */ private OffsetCommitRequest(String groupId, int generationId, String memberId, long retentionTime, Map offsetData, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, version)), version); - initCommonFields(groupId, offsetData); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); - struct.set(RETENTION_TIME_KEY_NAME, retentionTime); + super(version); this.groupId = groupId; this.generationId = generationId; this.memberId = memberId; @@ -212,35 +162,8 @@ private OffsetCommitRequest(String groupId, int generationId, String memberId, l this.offsetData = offsetData; } - private void initCommonFields(String groupId, Map offsetData) { - Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); - - struct.set(GROUP_ID_KEY_NAME, groupId); - List topicArray = new ArrayList(); - - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - // Only for v1 - if (partitionData.hasField(TIMESTAMP_KEY_NAME)) - partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); - partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - } - public OffsetCommitRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); groupId = struct.getString(GROUP_ID_KEY_NAME); // This field only exists in v1. @@ -283,6 +206,42 @@ public OffsetCommitRequest(Struct struct, short versionId) { } } + @Override + public Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, version)); + struct.set(GROUP_ID_KEY_NAME, groupId); + + Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); + List topicArray = new ArrayList<>(); + for (Map.Entry> topicEntry: topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); + // Only for v1 + if (partitionData.hasField(TIMESTAMP_KEY_NAME)) + partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); + partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + if (struct.hasField(GENERATION_ID_KEY_NAME)) + struct.set(GENERATION_ID_KEY_NAME, generationId); + if (struct.hasField(MEMBER_ID_KEY_NAME)) + struct.set(MEMBER_ID_KEY_NAME, memberId); + if (struct.hasField(RETENTION_TIME_KEY_NAME)) + struct.set(RETENTION_TIME_KEY_NAME, retentionTime); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { Map responseData = new HashMap<>(); @@ -322,12 +281,8 @@ public Map offsetData() { return offsetData; } - public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) { + public static OffsetCommitRequest parse(ByteBuffer buffer, short versionId) { Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId); - return new OffsetCommitRequest(schema.read(buffer), (short) versionId); - } - - public static OffsetCommitRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id)); + return new OffsetCommitRequest(schema.read(buffer), versionId); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 8a00c6bcc4f3e..b5709e2dada45 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; @@ -28,7 +27,6 @@ public class OffsetCommitResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); private static final String RESPONSES_KEY_NAME = "responses"; // topic level fields @@ -58,15 +56,33 @@ public class OffsetCommitResponse extends AbstractResponse { private final Map responseData; public OffsetCommitResponse(Map responseData) { - super(new Struct(CURRENT_SCHEMA)); + this.responseData = responseData; + } - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + public OffsetCommitResponse(Struct struct) { + responseData = new HashMap<>(); + for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { + Struct topicResponse = (Struct) topicResponseObj; + String topic = topicResponse.getString(TOPIC_KEY_NAME); + for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionResponse = (Struct) partitionResponseObj; + int partition = partitionResponse.getInt(PARTITION_KEY_NAME); + Errors error = Errors.forCode(partitionResponse.getShort(ERROR_CODE_KEY_NAME)); + responseData.put(new TopicPartition(topic, partition), error); + } + } + } + + @Override + public Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_COMMIT.id, version)); - List topicArray = new ArrayList(); + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List topicArray = new ArrayList<>(); for (Map.Entry> entries: topicsData.entrySet()) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList(); + List partitionArray = new ArrayList<>(); for (Map.Entry partitionEntry : entries.getValue().entrySet()) { Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); @@ -77,34 +93,16 @@ public OffsetCommitResponse(Map responseData) { topicArray.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); - this.responseData = responseData; - } - public OffsetCommitResponse(Struct struct) { - super(struct); - responseData = new HashMap(); - for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - Errors error = Errors.forCode(partitionResponse.getShort(ERROR_CODE_KEY_NAME)); - responseData.put(new TopicPartition(topic, partition), error); - } - } + return struct; } public Map responseData() { return responseData; } - public static OffsetCommitResponse parse(ByteBuffer buffer, int version) { - Schema schema = ProtoUtils.responseSchema(ApiKeys.OFFSET_COMMIT.id, version); - return new OffsetCommitResponse(schema.read(buffer)); + public static OffsetCommitResponse parse(ByteBuffer buffer, short version) { + return new OffsetCommitResponse(ProtoUtils.parseResponse(ApiKeys.OFFSET_COMMIT.id, version, buffer)); } - public static OffsetCommitResponse parse(ByteBuffer buffer) { - return new OffsetCommitResponse(CURRENT_SCHEMA.read(buffer)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 43ddf88d02b4f..2a550e508236d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -58,11 +58,11 @@ public boolean isAllTopicPartitions() { } @Override - public OffsetFetchRequest build() { - if (isAllTopicPartitions() && version() < 2) + public OffsetFetchRequest build(short version) { + if (isAllTopicPartitions() && version < 2) throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " + - "v" + version() + ", but we need v2 or newer to request all topic partitions."); - return new OffsetFetchRequest(groupId, partitions, version()); + "v" + version + ", but we need v2 or newer to request all topic partitions."); + return new OffsetFetchRequest(groupId, partitions, version); } @Override @@ -80,39 +80,18 @@ public String toString() { private final List partitions; public static OffsetFetchRequest forAllPartitions(String groupId) { - return new OffsetFetchRequest.Builder(groupId, null).setVersion((short) 2).build(); + return new OffsetFetchRequest.Builder(groupId, null).build((short) 2); } // v0, v1, and v2 have the same fields. private OffsetFetchRequest(String groupId, List partitions, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_FETCH.id, version)), version); - struct.set(GROUP_ID_KEY_NAME, groupId); - if (partitions != null) { - Map> topicsData = CollectionUtils.groupDataByTopic(partitions); - - List topicArray = new ArrayList<>(); - for (Map.Entry> entries : topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList<>(); - for (Integer partitionId : entries.getValue()) { - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionId); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - } else - struct.set(TOPICS_KEY_NAME, null); - + super(version); this.groupId = groupId; this.partitions = partitions; } - public OffsetFetchRequest(Struct struct, short versionId) { - super(struct, versionId); + public OffsetFetchRequest(Struct struct, short version) { + super(version); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); if (topicArray != null) { @@ -150,7 +129,7 @@ public OffsetFetchResponse getErrorResponse(Errors error) { case 0: case 1: case 2: - return new OffsetFetchResponse(error, responsePartitions, versionId); + return new OffsetFetchResponse(error, responsePartitions); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); @@ -170,16 +149,38 @@ public List partitions() { return partitions; } - public static OffsetFetchRequest parse(ByteBuffer buffer, int versionId) { - return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer), - (short) versionId); - } - - public static OffsetFetchRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id)); + public static OffsetFetchRequest parse(ByteBuffer buffer, short versionId) { + return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer), versionId); } public boolean isAllPartitions() { return partitions == null; } + + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_FETCH.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + if (partitions != null) { + Map> topicsData = CollectionUtils.groupDataByTopic(partitions); + + List topicArray = new ArrayList<>(); + for (Map.Entry> entries : topicsData.entrySet()) { + Struct topicData = struct.instance(TOPICS_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + List partitionArray = new ArrayList<>(); + for (Integer partitionId : entries.getValue()) { + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionId); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(TOPICS_KEY_NAME, topicArray.toArray()); + } else + struct.set(TOPICS_KEY_NAME, null); + + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 9c1415538c54f..94de4b11a88c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -24,14 +24,11 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; public class OffsetFetchResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); - private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id); private static final String RESPONSES_KEY_NAME = "responses"; private static final String ERROR_CODE_KEY_NAME = "error_code"; @@ -85,51 +82,16 @@ public boolean hasError() { } /** - * Constructor for the latest version. - * @param error Potential coordinator or group level error code - * @param responseData Fetched offset information grouped by topic-partition - */ - public OffsetFetchResponse(Errors error, Map responseData) { - this(error, responseData, CURRENT_VERSION); - } - - /** - * Unified constructor for all versions. + * Constructor for all versions. * @param error Potential coordinator or group level error code (for api version 2 and later) * @param responseData Fetched offset information grouped by topic-partition - * @param version The request API version */ - public OffsetFetchResponse(Errors error, Map responseData, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); - - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - List topicArray = new ArrayList<>(); - for (Map.Entry> entries : topicsData.entrySet()) { - Struct topicData = this.struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : entries.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); - partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.error.code()); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - - this.struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + public OffsetFetchResponse(Errors error, Map responseData) { this.responseData = responseData; this.error = error; - if (version > 1) - this.struct.set(ERROR_CODE_KEY_NAME, this.error.code()); } public OffsetFetchResponse(Struct struct) { - super(struct); Errors topLevelError = Errors.NONE; this.responseData = new HashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { @@ -175,12 +137,37 @@ public Map responseData() { return responseData; } - public static OffsetFetchResponse parse(ByteBuffer buffer, int version) { - Schema schema = ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version); - return new OffsetFetchResponse(schema.read(buffer)); + public static OffsetFetchResponse parse(ByteBuffer buffer, short version) { + return new OffsetFetchResponse(ProtoUtils.parseResponse(ApiKeys.OFFSET_FETCH.id, version, buffer)); } - public static OffsetFetchResponse parse(ByteBuffer buffer) { - return new OffsetFetchResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version)); + + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List topicArray = new ArrayList<>(); + for (Map.Entry> entries : topicsData.entrySet()) { + Struct topicData = struct.instance(RESPONSES_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entries.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : entries.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); + Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); + partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); + partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); + partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.error.code()); + partitionArray.add(partitionData); + } + topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); + topicArray.add(topicData); + } + struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + + if (version > 1) + struct.set(ERROR_CODE_KEY_NAME, this.error.code()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index bd3ae8ff3b531..df70e20bd6f42 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -55,11 +55,10 @@ public Builder(short acks, int timeout, Map parti } @Override - public ProduceRequest build() { - short version = version(); - if (version < 2) { + public ProduceRequest build(short version) { + if (version < 2) throw new UnsupportedVersionException("ProduceRequest versions older than 2 are not supported."); - } + return new ProduceRequest(version, acks, timeout, partitionRecords); } @@ -80,33 +79,14 @@ public String toString() { private final Map partitionRecords; private ProduceRequest(short version, short acks, int timeout, Map partitionRecords) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, version)), version); - Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); - struct.set(ACKS_KEY_NAME, acks); - struct.set(TIMEOUT_KEY_NAME, timeout); - List topicDatas = new ArrayList<>(recordsByTopic.size()); - for (Map.Entry> entry : recordsByTopic.entrySet()) { - Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entry.getKey()); - List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : entry.getValue().entrySet()) { - MemoryRecords records = partitionEntry.getValue(); - Struct part = topicData.instance(PARTITION_DATA_KEY_NAME) - .set(PARTITION_KEY_NAME, partitionEntry.getKey()) - .set(RECORD_SET_KEY_NAME, records); - partitionArray.add(part); - } - topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray()); - topicDatas.add(topicData); - } - struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray()); + super(version); this.acks = acks; this.timeout = timeout; this.partitionRecords = partitionRecords; } public ProduceRequest(Struct struct, short version) { - super(struct, version); + super(version); partitionRecords = new HashMap<>(); for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) { Struct topicData = (Struct) topicDataObj; @@ -122,6 +102,34 @@ public ProduceRequest(Struct struct, short version) { timeout = struct.getInt(TIMEOUT_KEY_NAME); } + /** + * Visible for testing. + */ + @Override + public Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, version())); + Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); + struct.set(ACKS_KEY_NAME, acks); + struct.set(TIMEOUT_KEY_NAME, timeout); + List topicDatas = new ArrayList<>(recordsByTopic.size()); + for (Map.Entry> entry : recordsByTopic.entrySet()) { + Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, entry.getKey()); + List partitionArray = new ArrayList<>(); + for (Map.Entry partitionEntry : entry.getValue().entrySet()) { + MemoryRecords records = partitionEntry.getValue(); + Struct part = topicData.instance(PARTITION_DATA_KEY_NAME) + .set(PARTITION_KEY_NAME, partitionEntry.getKey()) + .set(RECORD_SET_KEY_NAME, records); + partitionArray.add(part); + } + topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray()); + topicDatas.add(topicData); + } + struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray()); + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { /* In case the producer doesn't actually want any response */ @@ -137,10 +145,9 @@ public AbstractResponse getErrorResponse(Throwable e) { short versionId = version(); switch (versionId) { case 0: - return new ProduceResponse(responseMap); case 1: case 2: - return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME, versionId); + return new ProduceResponse(responseMap); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id))); @@ -160,15 +167,10 @@ public Map partitionRecords() { } public void clearPartitionRecords() { - struct.clear(); partitionRecords.clear(); } - public static ProduceRequest parse(ByteBuffer buffer, int versionId) { - return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer), (short) versionId); - } - - public static ProduceRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)); + public static ProduceRequest parse(ByteBuffer buffer, short versionId) { + return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer), versionId); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 9eaaadf617e15..7a022affc340a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.utils.CollectionUtils; @@ -31,8 +30,7 @@ * This wrapper supports both v0 and v1 of ProduceResponse. */ public class ProduceResponse extends AbstractResponse { - - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names @@ -73,10 +71,7 @@ public class ProduceResponse extends AbstractResponse { * @param responses Produced data grouped by topic-partition */ public ProduceResponse(Map responses) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0))); - initCommonFields(responses); - this.responses = responses; - this.throttleTime = DEFAULT_THROTTLE_TIME; + this(responses, DEFAULT_THROTTLE_TIME); } /** @@ -85,30 +80,14 @@ public ProduceResponse(Map responses) { * @param throttleTime Time in milliseconds the response was throttled */ public ProduceResponse(Map responses, int throttleTime) { - this(responses, throttleTime, ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)); - } - - /** - * Constructor for a specific version - * @param responses Produced data grouped by topic-partition - * @param throttleTime Time in milliseconds the response was throttled - * @param version the version of schema to use. - */ - public ProduceResponse(Map responses, int throttleTime, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version))); - initCommonFields(responses); - if (struct.hasField(THROTTLE_TIME_KEY_NAME)) - struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); this.responses = responses; this.throttleTime = throttleTime; } /** - * Constructor from a {@link Struct}. It is the caller's responsibility to pass in a struct with the latest schema. - * @param struct + * Constructor from a {@link Struct}. */ public ProduceResponse(Struct struct) { - super(struct); responses = new HashMap<>(); for (Object topicResponse : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicRespStruct = (Struct) topicResponse; @@ -126,7 +105,10 @@ public ProduceResponse(Struct struct) { this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); } - private void initCommonFields(Map responses) { + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version)); + Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList<>(responseByTopic.size()); for (Map.Entry> entry : responseByTopic.entrySet()) { @@ -140,13 +122,17 @@ private void initCommonFields(Map responses) .set(ERROR_CODE_KEY_NAME, part.error.code()) .set(BASE_OFFSET_KEY_NAME, part.baseOffset); if (partStruct.hasField(LOG_APPEND_TIME_KEY_NAME)) - partStruct.set(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime); + partStruct.set(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime); partitionArray.add(partStruct); } topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); topicDatas.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); + + if (struct.hasField(THROTTLE_TIME_KEY_NAME)) + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); + return struct; } public Map responses() { @@ -187,7 +173,7 @@ public String toString() { } } - public static ProduceResponse parse(ByteBuffer buffer) { - return new ProduceResponse(CURRENT_SCHEMA.read(buffer)); + public static ProduceResponse parse(ByteBuffer buffer, short version) { + return new ProduceResponse(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java new file mode 100644 index 0000000000000..d2147b3181400 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +public class RequestAndSize { + public final AbstractRequest request; + public final int size; + + public RequestAndSize(AbstractRequest request, int size) { + this.request = request; + this.size = size; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java index 05b78cbe9e1d7..5e65132449ff3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java @@ -35,8 +35,7 @@ public class RequestHeader extends AbstractRequestResponse { private final String clientId; private final int correlationId; - public RequestHeader(Struct header) { - super(header); + public RequestHeader(Struct struct) { apiKey = struct.getShort(API_KEY_FIELD); apiVersion = struct.getShort(API_VERSION_FIELD); clientId = struct.getString(CLIENT_ID_FIELD); @@ -44,17 +43,21 @@ public RequestHeader(Struct header) { } public RequestHeader(short apiKey, short version, String client, int correlation) { - super(new Struct(Protocol.REQUEST_HEADER)); - struct.set(API_KEY_FIELD, apiKey); - struct.set(API_VERSION_FIELD, version); - struct.set(CLIENT_ID_FIELD, client); - struct.set(CORRELATION_ID_FIELD, correlation); this.apiKey = apiKey; this.apiVersion = version; this.clientId = client; this.correlationId = correlation; } + public Struct toStruct() { + Struct struct = new Struct(Protocol.REQUEST_HEADER); + struct.set(API_KEY_FIELD, apiKey); + struct.set(API_VERSION_FIELD, apiVersion); + struct.set(CLIENT_ID_FIELD, clientId); + struct.set(CORRELATION_ID_FIELD, correlationId); + return struct; + } + public short apiKey() { return apiKey; } @@ -71,6 +74,10 @@ public int correlationId() { return correlationId; } + public ResponseHeader toResponseHeader() { + return new ResponseHeader(correlationId); + } + public static RequestHeader parse(ByteBuffer buffer) { return new RequestHeader(Protocol.REQUEST_HEADER.read(buffer)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java index e68bd39a7e3c1..04390eaae34eb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java @@ -34,17 +34,24 @@ public class ResponseHeader extends AbstractRequestResponse { private final int correlationId; - public ResponseHeader(Struct header) { - super(header); + public ResponseHeader(Struct struct) { correlationId = struct.getInt(CORRELATION_KEY_FIELD); } public ResponseHeader(int correlationId) { - super(new Struct(Protocol.RESPONSE_HEADER)); - struct.set(CORRELATION_KEY_FIELD, correlationId); this.correlationId = correlationId; } + public int sizeOf() { + return toStruct().sizeOf(); + } + + public Struct toStruct() { + Struct struct = new Struct(Protocol.RESPONSE_HEADER); + struct.set(CORRELATION_KEY_FIELD, correlationId); + return struct; + } + public int correlationId() { return correlationId; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java index d244f0af682c1..a1f3f0efd35ad 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; @@ -40,19 +39,17 @@ */ public class SaslHandshakeRequest extends AbstractRequest { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.SASL_HANDSHAKE.id); public static final String MECHANISM_KEY_NAME = "mechanism"; private final String mechanism; public SaslHandshakeRequest(String mechanism) { - super(new Struct(CURRENT_SCHEMA), ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)); - struct.set(MECHANISM_KEY_NAME, mechanism); + super(ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)); this.mechanism = mechanism; } - public SaslHandshakeRequest(Struct struct, short versionId) { - super(struct, versionId); + public SaslHandshakeRequest(Struct struct, short version) { + super(version); mechanism = struct.getString(MECHANISM_KEY_NAME); } @@ -73,13 +70,15 @@ public AbstractResponse getErrorResponse(Throwable e) { } } - public static SaslHandshakeRequest parse(ByteBuffer buffer, int versionId) { - return new SaslHandshakeRequest(ProtoUtils.parseRequest(ApiKeys.SASL_HANDSHAKE.id, versionId, buffer), - (short) versionId); + public static SaslHandshakeRequest parse(ByteBuffer buffer, short versionId) { + return new SaslHandshakeRequest(ProtoUtils.parseRequest(ApiKeys.SASL_HANDSHAKE.id, versionId, buffer), versionId); } - public static SaslHandshakeRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.SASL_HANDSHAKE.id, version())); + struct.set(MECHANISM_KEY_NAME, mechanism); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java index f50c5becf4ce3..9d38c6ab21a3f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; @@ -36,8 +35,6 @@ */ public class SaslHandshakeResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.SASL_HANDSHAKE.id); - private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String ENABLED_MECHANISMS_KEY_NAME = "enabled_mechanisms"; @@ -50,15 +47,11 @@ public class SaslHandshakeResponse extends AbstractResponse { private final List enabledMechanisms; public SaslHandshakeResponse(Errors error, Collection enabledMechanisms) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray()); this.error = error; this.enabledMechanisms = new ArrayList<>(enabledMechanisms); } public SaslHandshakeResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); Object[] mechanisms = struct.getArray(ENABLED_MECHANISMS_KEY_NAME); ArrayList enabledMechanisms = new ArrayList<>(); @@ -71,15 +64,19 @@ public Errors error() { return error; } - public List enabledMechanisms() { - return enabledMechanisms; + @Override + public Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.SASL_HANDSHAKE.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray()); + return struct; } - public static SaslHandshakeResponse parse(ByteBuffer buffer) { - return new SaslHandshakeResponse(CURRENT_SCHEMA.read(buffer)); + public List enabledMechanisms() { + return enabledMechanisms; } - public static SaslHandshakeResponse parse(ByteBuffer buffer, int version) { + public static SaslHandshakeResponse parse(ByteBuffer buffer, short version) { return new SaslHandshakeResponse(ProtoUtils.parseResponse(ApiKeys.SASL_HANDSHAKE.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java index ff2638b01a2bb..91806f18f2b48 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -52,9 +52,9 @@ public Builder(int controllerId, int controllerEpoch, boolean deletePartitions, } @Override - public StopReplicaRequest build() { + public StopReplicaRequest build(short version) { return new StopReplicaRequest(controllerId, controllerEpoch, - deletePartitions, partitions, version()); + deletePartitions, partitions, version); } @Override @@ -77,30 +77,15 @@ public String toString() { private StopReplicaRequest(int controllerId, int controllerEpoch, boolean deletePartitions, Set partitions, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.STOP_REPLICA.id, version)), version); - - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); - struct.set(DELETE_PARTITIONS_KEY_NAME, deletePartitions); - - List partitionDatas = new ArrayList<>(partitions.size()); - for (TopicPartition partition : partitions) { - Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); - partitionData.set(TOPIC_KEY_NAME, partition.topic()); - partitionData.set(PARTITION_KEY_NAME, partition.partition()); - partitionDatas.add(partitionData); - } - - struct.set(PARTITIONS_KEY_NAME, partitionDatas.toArray()); - + super(version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.deletePartitions = deletePartitions; this.partitions = partitions; } - public StopReplicaRequest(Struct struct, short versionId) { - super(struct, versionId); + public StopReplicaRequest(Struct struct, short version) { + super(version); partitions = new HashSet<>(); for (Object partitionDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { @@ -148,12 +133,27 @@ public Set partitions() { return partitions; } - public static StopReplicaRequest parse(ByteBuffer buffer, int versionId) { - return new StopReplicaRequest(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer), - (short) versionId); + public static StopReplicaRequest parse(ByteBuffer buffer, short versionId) { + return new StopReplicaRequest(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer), versionId); } - public static StopReplicaRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.STOP_REPLICA.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.STOP_REPLICA.id, version())); + + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + struct.set(DELETE_PARTITIONS_KEY_NAME, deletePartitions); + + List partitionDatas = new ArrayList<>(partitions.size()); + for (TopicPartition partition : partitions) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + partitionData.set(TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITION_KEY_NAME, partition.partition()); + partitionDatas.add(partitionData); + } + + struct.set(PARTITIONS_KEY_NAME, partitionDatas.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java index b39fb193c511f..5ae5cc1cab5be 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -17,7 +17,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -27,7 +26,6 @@ import java.util.Map; public class StopReplicaResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.STOP_REPLICA.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; private static final String PARTITIONS_KEY_NAME = "partitions"; @@ -37,41 +35,20 @@ public class StopReplicaResponse extends AbstractResponse { private static final String PARTITIONS_ERROR_CODE_KEY_NAME = "error_code"; private final Map responses; - private final Errors error; /** * Possible error code: * * STALE_CONTROLLER_EPOCH (11) */ - - public StopReplicaResponse(Map responses) { - this(Errors.NONE, responses); - } + private final Errors error; public StopReplicaResponse(Errors error, Map responses) { - super(new Struct(CURRENT_SCHEMA)); - - List responseDatas = new ArrayList<>(responses.size()); - for (Map.Entry response : responses.entrySet()) { - Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); - TopicPartition partition = response.getKey(); - partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); - partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); - partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); - responseDatas.add(partitionData); - } - - struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); - struct.set(ERROR_CODE_KEY_NAME, error.code()); - this.responses = responses; this.error = error; } public StopReplicaResponse(Struct struct) { - super(struct); - responses = new HashMap<>(); for (Object responseDataObj : struct.getArray(PARTITIONS_KEY_NAME)) { Struct responseData = (Struct) responseDataObj; @@ -92,11 +69,27 @@ public Errors error() { return error; } - public static StopReplicaResponse parse(ByteBuffer buffer, int versionId) { - return new StopReplicaResponse(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer)); + public static StopReplicaResponse parse(ByteBuffer buffer, short versionId) { + return new StopReplicaResponse(ProtoUtils.parseResponse(ApiKeys.STOP_REPLICA.id, versionId, buffer)); } - public static StopReplicaResponse parse(ByteBuffer buffer) { - return new StopReplicaResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.STOP_REPLICA.id, version)); + + List responseDatas = new ArrayList<>(responses.size()); + for (Map.Entry response : responses.entrySet()) { + Struct partitionData = struct.instance(PARTITIONS_KEY_NAME); + TopicPartition partition = response.getKey(); + partitionData.set(PARTITIONS_TOPIC_KEY_NAME, partition.topic()); + partitionData.set(PARTITIONS_PARTITION_KEY_NAME, partition.partition()); + partitionData.set(PARTITIONS_ERROR_CODE_KEY_NAME, response.getValue().code()); + responseDatas.add(partitionData); + } + + struct.set(PARTITIONS_KEY_NAME, responseDatas.toArray()); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java index 937bf98213d7b..7ad5c9a168d68 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java @@ -51,8 +51,8 @@ public Builder(String groupId, int generationId, String memberId, } @Override - public SyncGroupRequest build() { - return new SyncGroupRequest(groupId, generationId, memberId, groupAssignment, version()); + public SyncGroupRequest build(short version) { + return new SyncGroupRequest(groupId, generationId, memberId, groupAssignment, version); } @Override @@ -75,20 +75,7 @@ public String toString() { private SyncGroupRequest(String groupId, int generationId, String memberId, Map groupAssignment, short version) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.SYNC_GROUP.id, version)), version); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(MEMBER_ID_KEY_NAME, memberId); - - List memberArray = new ArrayList<>(); - for (Map.Entry entries: groupAssignment.entrySet()) { - Struct memberData = struct.instance(GROUP_ASSIGNMENT_KEY_NAME); - memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); - memberData.set(MEMBER_ASSIGNMENT_KEY_NAME, entries.getValue()); - memberArray.add(memberData); - } - struct.set(GROUP_ASSIGNMENT_KEY_NAME, memberArray.toArray()); - + super(version); this.groupId = groupId; this.generationId = generationId; this.memberId = memberId; @@ -96,7 +83,7 @@ private SyncGroupRequest(String groupId, int generationId, String memberId, } public SyncGroupRequest(Struct struct, short version) { - super(struct, version); + super(version); this.groupId = struct.getString(GROUP_ID_KEY_NAME); this.generationId = struct.getInt(GENERATION_ID_KEY_NAME); this.memberId = struct.getString(MEMBER_ID_KEY_NAME); @@ -141,12 +128,25 @@ public String memberId() { return memberId; } - public static SyncGroupRequest parse(ByteBuffer buffer, int versionId) { - return new SyncGroupRequest(ProtoUtils.parseRequest(ApiKeys.SYNC_GROUP.id, versionId, buffer), - (short) versionId); + public static SyncGroupRequest parse(ByteBuffer buffer, short versionId) { + return new SyncGroupRequest(ProtoUtils.parseRequest(ApiKeys.SYNC_GROUP.id, versionId, buffer), versionId); } - public static SyncGroupRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.SYNC_GROUP.id)); + @Override + protected Struct toStruct() { + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.SYNC_GROUP.id, version())); + struct.set(GROUP_ID_KEY_NAME, groupId); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(MEMBER_ID_KEY_NAME, memberId); + + List memberArray = new ArrayList<>(); + for (Map.Entry entries: groupAssignment.entrySet()) { + Struct memberData = struct.instance(GROUP_ASSIGNMENT_KEY_NAME); + memberData.set(MEMBER_ID_KEY_NAME, entries.getKey()); + memberData.set(MEMBER_ASSIGNMENT_KEY_NAME, entries.getValue()); + memberArray.add(memberData); + } + struct.set(GROUP_ASSIGNMENT_KEY_NAME, memberArray.toArray()); + return struct; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java index e598975187e89..ff198aa0063b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java @@ -19,14 +19,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class SyncGroupResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.SYNC_GROUP.id); public static final String ERROR_CODE_KEY_NAME = "error_code"; public static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment"; @@ -45,18 +43,11 @@ public class SyncGroupResponse extends AbstractResponse { private final ByteBuffer memberState; public SyncGroupResponse(Errors error, ByteBuffer memberState) { - super(new Struct(CURRENT_SCHEMA)); - - struct.set(ERROR_CODE_KEY_NAME, error.code()); - struct.set(MEMBER_ASSIGNMENT_KEY_NAME, memberState); - this.error = error; this.memberState = memberState; } public SyncGroupResponse(Struct struct) { - super(struct); - this.error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); this.memberState = struct.getBytes(MEMBER_ASSIGNMENT_KEY_NAME); } @@ -69,8 +60,16 @@ public ByteBuffer memberAssignment() { return memberState; } - public static SyncGroupResponse parse(ByteBuffer buffer) { - return new SyncGroupResponse(CURRENT_SCHEMA.read(buffer)); + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.SYNC_GROUP.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + struct.set(MEMBER_ASSIGNMENT_KEY_NAME, memberState); + return struct; + } + + public static SyncGroupResponse parse(ByteBuffer buffer, short version) { + return new SyncGroupResponse(ProtoUtils.parseResponse(ApiKeys.SYNC_GROUP.id, version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index ef680ff3d6904..8dd852da6bee1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -38,10 +38,9 @@ public static class Builder extends AbstractRequest.Builder partitionStates; private final Set liveBrokers; - public Builder(int controllerId, int controllerEpoch, - Map partitionStates, - Set liveBrokers) { - super(ApiKeys.UPDATE_METADATA_KEY); + public Builder(short version, int controllerId, int controllerEpoch, + Map partitionStates, Set liveBrokers) { + super(ApiKeys.UPDATE_METADATA_KEY, version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.partitionStates = partitionStates; @@ -49,8 +48,7 @@ public Builder(int controllerId, int controllerEpoch, } @Override - public UpdateMetadataRequest build() { - short version = version(); + public UpdateMetadataRequest build(short version) { if (version == 0) { for (Broker broker : liveBrokers) { if (broker.endPoints.size() != 1 || broker.endPoints.get(0).securityProtocol != SecurityProtocol.PLAINTEXT) { @@ -148,58 +146,7 @@ public String toString() { private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch, Map partitionStates, Set liveBrokers) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)), version); - struct.set(CONTROLLER_ID_KEY_NAME, controllerId); - struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); - - List partitionStatesData = new ArrayList<>(partitionStates.size()); - for (Map.Entry entry : partitionStates.entrySet()) { - Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); - TopicPartition topicPartition = entry.getKey(); - partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); - partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); - PartitionState partitionState = entry.getValue(); - partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); - partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); - partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); - partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); - partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); - partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); - partitionStatesData.add(partitionStateData); - } - struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); - - List brokersData = new ArrayList<>(liveBrokers.size()); - for (Broker broker : liveBrokers) { - Struct brokerData = struct.instance(LIVE_BROKERS_KEY_NAME); - brokerData.set(BROKER_ID_KEY_NAME, broker.id); - - if (version == 0) { - EndPoint endPoint = broker.endPoints.get(0); - brokerData.set(HOST_KEY_NAME, endPoint.host); - brokerData.set(PORT_KEY_NAME, endPoint.port); - } else { - List endPointsData = new ArrayList<>(broker.endPoints.size()); - for (EndPoint endPoint : broker.endPoints) { - Struct endPointData = brokerData.instance(ENDPOINTS_KEY_NAME); - endPointData.set(PORT_KEY_NAME, endPoint.port); - endPointData.set(HOST_KEY_NAME, endPoint.host); - endPointData.set(SECURITY_PROTOCOL_TYPE_KEY_NAME, endPoint.securityProtocol.id); - if (version >= 3) - endPointData.set(LISTENER_NAME_KEY_NAME, endPoint.listenerName.value()); - endPointsData.add(endPointData); - - } - brokerData.set(ENDPOINTS_KEY_NAME, endPointsData.toArray()); - if (version >= 2) { - brokerData.set(RACK_KEY_NAME, broker.rack); - } - } - - brokersData.add(brokerData); - } - struct.set(LIVE_BROKERS_KEY_NAME, brokersData.toArray()); - + super(version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.partitionStates = partitionStates; @@ -207,7 +154,7 @@ private UpdateMetadataRequest(short version, int controllerId, int controllerEpo } public UpdateMetadataRequest(Struct struct, short versionId) { - super(struct, versionId); + super(versionId); Map partitionStates = new HashMap<>(); for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) { Struct partitionStateData = (Struct) partitionStateDataObj; @@ -276,6 +223,64 @@ public UpdateMetadataRequest(Struct struct, short versionId) { this.liveBrokers = liveBrokers; } + @Override + protected Struct toStruct() { + short version = version(); + Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)); + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); + + List partitionStatesData = new ArrayList<>(partitionStates.size()); + for (Map.Entry entry : partitionStates.entrySet()) { + Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); + TopicPartition topicPartition = entry.getKey(); + partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); + partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); + PartitionState partitionState = entry.getValue(); + partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); + partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); + partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); + partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); + partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); + partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + partitionStatesData.add(partitionStateData); + } + struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); + + List brokersData = new ArrayList<>(liveBrokers.size()); + for (Broker broker : liveBrokers) { + Struct brokerData = struct.instance(LIVE_BROKERS_KEY_NAME); + brokerData.set(BROKER_ID_KEY_NAME, broker.id); + + if (version == 0) { + EndPoint endPoint = broker.endPoints.get(0); + brokerData.set(HOST_KEY_NAME, endPoint.host); + brokerData.set(PORT_KEY_NAME, endPoint.port); + } else { + List endPointsData = new ArrayList<>(broker.endPoints.size()); + for (EndPoint endPoint : broker.endPoints) { + Struct endPointData = brokerData.instance(ENDPOINTS_KEY_NAME); + endPointData.set(PORT_KEY_NAME, endPoint.port); + endPointData.set(HOST_KEY_NAME, endPoint.host); + endPointData.set(SECURITY_PROTOCOL_TYPE_KEY_NAME, endPoint.securityProtocol.id); + if (version >= 3) + endPointData.set(LISTENER_NAME_KEY_NAME, endPoint.listenerName.value()); + endPointsData.add(endPointData); + + } + brokerData.set(ENDPOINTS_KEY_NAME, endPointsData.toArray()); + if (version >= 2) { + brokerData.set(RACK_KEY_NAME, broker.rack); + } + } + + brokersData.add(brokerData); + } + struct.set(LIVE_BROKERS_KEY_NAME, brokersData.toArray()); + + return struct; + } + @Override public AbstractResponse getErrorResponse(Throwable e) { short versionId = version(); @@ -302,12 +307,9 @@ public Set liveBrokers() { return liveBrokers; } - public static UpdateMetadataRequest parse(ByteBuffer buffer, int versionId) { + public static UpdateMetadataRequest parse(ByteBuffer buffer, short versionId) { return new UpdateMetadataRequest(ProtoUtils.parseRequest(ApiKeys.UPDATE_METADATA_KEY.id, versionId, buffer), - (short) versionId); + versionId); } - public static UpdateMetadataRequest parse(ByteBuffer buffer) { - return parse(buffer, ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id)); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java index 865d6c608c5a7..0032fca77a7d4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java @@ -16,15 +16,12 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; public class UpdateMetadataResponse extends AbstractResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.UPDATE_METADATA_KEY.id); - private static final String ERROR_CODE_KEY_NAME = "error_code"; /** @@ -35,13 +32,10 @@ public class UpdateMetadataResponse extends AbstractResponse { private final Errors error; public UpdateMetadataResponse(Errors error) { - super(new Struct(CURRENT_SCHEMA)); - struct.set(ERROR_CODE_KEY_NAME, error.code()); this.error = error; } public UpdateMetadataResponse(Struct struct) { - super(struct); error = Errors.forCode(struct.getShort(ERROR_CODE_KEY_NAME)); } @@ -49,12 +43,14 @@ public Errors error() { return error; } - public static UpdateMetadataResponse parse(ByteBuffer buffer) { - return new UpdateMetadataResponse(CURRENT_SCHEMA.read(buffer)); - } - - public static UpdateMetadataResponse parse(ByteBuffer buffer, int version) { + public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) { return new UpdateMetadataResponse(ProtoUtils.parseResponse(ApiKeys.UPDATE_METADATA_KEY.id, version, buffer)); } + @Override + protected Struct toStruct(short version) { + Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)); + struct.set(ERROR_CODE_KEY_NAME, error.code()); + return struct; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index 7f6b7aaa5993f..88f8959282611 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -308,11 +308,12 @@ private boolean handleKafkaRequest(byte[] requestBytes) throws IOException, Auth if (!Protocol.apiVersionSupported(requestHeader.apiKey(), requestHeader.apiVersion())) { if (apiKey == ApiKeys.API_VERSIONS) - sendKafkaResponse(requestHeader, ApiVersionsResponse.fromError(Errors.UNSUPPORTED_VERSION)); + sendKafkaResponse(ApiVersionsResponse.unsupportedVersionSend(node, requestHeader)); else throw new UnsupportedVersionException("Version " + requestHeader.apiVersion() + " is not supported for apiKey " + apiKey); } else { - AbstractRequest request = AbstractRequest.getRequest(requestHeader.apiKey(), requestHeader.apiVersion(), requestBuffer); + AbstractRequest request = AbstractRequest.getRequest(requestHeader.apiKey(), requestHeader.apiVersion(), + requestBuffer).request; LOG.debug("Handle Kafka request {}", apiKey); switch (apiKey) { @@ -373,7 +374,11 @@ private void handleApiVersionsRequest(RequestHeader requestHeader) throws IOExce } private void sendKafkaResponse(RequestHeader requestHeader, AbstractResponse response) throws IOException { - netOutBuffer = response.toSend(node, requestHeader); + sendKafkaResponse(response.toSend(node, requestHeader)); + } + + private void sendKafkaResponse(Send send) throws IOException { + netOutBuffer = send; flushNetOutBufferAndUpdateInterestOps(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java index ba3863734391b..106a7d47f78b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java @@ -26,14 +26,14 @@ public class CollectionUtils { * @param Partition data type * @return partitioned data */ - public static Map> groupDataByTopic(Map data) { - Map> dataByTopic = new HashMap>(); - for (Map.Entry entry: data.entrySet()) { + public static Map> groupDataByTopic(Map data) { + Map> dataByTopic = new HashMap<>(); + for (Map.Entry entry: data.entrySet()) { String topic = entry.getKey().topic(); int partition = entry.getKey().partition(); Map topicData = dataByTopic.get(topic); if (topicData == null) { - topicData = new HashMap(); + topicData = new HashMap<>(); dataByTopic.put(topic, topicData); } topicData.put(partition, entry.getValue()); @@ -47,12 +47,12 @@ public static Map> groupDataByTopic(Map> groupDataByTopic(List partitions) { - Map> partitionsByTopic = new HashMap>(); + Map> partitionsByTopic = new HashMap<>(); for (TopicPartition tp: partitions) { String topic = tp.topic(); List topicData = partitionsByTopic.get(topic); if (topicData == null) { - topicData = new ArrayList(); + topicData = new ArrayList<>(); partitionsByTopic.put(topic, topicData); } topicData.add(tp.partition()); diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 50ed1319cdac3..7712d3c8577b2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -131,7 +131,8 @@ public void disconnect(String node) { while (iter.hasNext()) { ClientRequest request = iter.next(); if (request.destination().equals(node)) { - responses.add(new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + short version = request.requestBuilder().desiredOrLatestVersion(); + responses.add(new ClientResponse(request.makeHeader(version), request.callback(), request.destination(), request.createdTimeMs(), now, true, null, null)); iter.remove(); } @@ -146,13 +147,11 @@ public void send(ClientRequest request, long now) { FutureResponse futureResp = iterator.next(); if (futureResp.node != null && !request.destination().equals(futureResp.node.idString())) continue; - request.requestBuilder().setVersion(nodeApiVersions.usableVersion( - request.requestBuilder().apiKey())); - AbstractRequest abstractRequest = request.requestBuilder().build(); + short usableVersion = nodeApiVersions.usableVersion(request.requestBuilder().apiKey()); + AbstractRequest abstractRequest = request.requestBuilder().build(usableVersion); if (!futureResp.requestMatcher.matches(abstractRequest)) throw new IllegalStateException("Next in line response did not match expected request"); - - ClientResponse resp = new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + ClientResponse resp = new ClientResponse(request.makeHeader(usableVersion), request.callback(), request.destination(), request.createdTimeMs(), time.milliseconds(), futureResp.disconnected, null, futureResp.responseBody); responses.add(resp); iterator.remove(); @@ -192,7 +191,8 @@ public void respond(AbstractResponse response) { public void respond(AbstractResponse response, boolean disconnected) { ClientRequest request = requests.remove(); - responses.add(new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + short version = request.requestBuilder().desiredOrLatestVersion(); + responses.add(new ClientResponse(request.makeHeader(version), request.callback(), request.destination(), request.createdTimeMs(), time.milliseconds(), disconnected, null, response)); } @@ -206,7 +206,8 @@ public void respondFrom(AbstractResponse response, Node node, boolean disconnect ClientRequest request = iterator.next(); if (request.destination().equals(node.idString())) { iterator.remove(); - responses.add(new ClientResponse(request.makeHeader(), request.callback(), request.destination(), + short version = request.requestBuilder().desiredOrLatestVersion(); + responses.add(new ClientResponse(request.makeHeader(version), request.callback(), request.destination(), request.createdTimeMs(), time.milliseconds(), disconnected, null, response)); return; } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index deaf2cc15e14e..c89cc24886bed 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.requests.AbstractRequestResponse; import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.ProduceRequest; @@ -137,9 +136,10 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { ResponseHeader respHeader = new ResponseHeader(request.correlationId()); Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); resp.set("responses", new Object[0]); - int size = respHeader.sizeOf() + resp.sizeOf(); + Struct responseHeaderStruct = respHeader.toStruct(); + int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); ByteBuffer buffer = ByteBuffer.allocate(size); - respHeader.writeTo(buffer); + responseHeaderStruct.writeTo(buffer); resp.writeTo(buffer); buffer.flip(); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); @@ -152,9 +152,7 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { } private void maybeSetExpectedApiVersionsResponse() { - ResponseHeader responseHeader = new ResponseHeader(0); - ByteBuffer buffer = AbstractRequestResponse.serialize(responseHeader, - ApiVersionsResponse.API_VERSIONS_RESPONSE); + ByteBuffer buffer = ApiVersionsResponse.API_VERSIONS_RESPONSE.serialize((short) 0, new ResponseHeader(0)); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index a355aa18d095d..5c4590b36e716 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1436,7 +1436,7 @@ private ListOffsetResponse listOffsetsResponse(Map offsets partitionData.put(partitionOffset.getKey(), new ListOffsetResponse.PartitionData(error, 1L, partitionOffset.getValue())); } - return new ListOffsetResponse(partitionData, 1); + return new ListOffsetResponse(partitionData); } private FetchResponse fetchResponse(Map fetches) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index fec92519efc56..6a17da801aac9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -886,7 +886,7 @@ private ListOffsetResponse listOffsetResponse(TopicPartition tp, Errors error, l ListOffsetResponse.PartitionData partitionData = new ListOffsetResponse.PartitionData(error, timestamp, offset); Map allPartitionData = new HashMap<>(); allPartitionData.put(tp, partitionData); - return new ListOffsetResponse(allPartitionData, 1); + return new ListOffsetResponse(allPartitionData); } private FetchResponse fetchResponse(MemoryRecords records, Errors error, long hw, int throttleTime) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 699f6e23c0f4f..d0b9639806bd7 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -30,6 +30,7 @@ import org.junit.Test; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.nio.channels.GatheringByteChannel; @@ -51,133 +52,161 @@ public class RequestResponseTest { @Test public void testSerialization() throws Exception { - checkSerialization(createRequestHeader(), null); - checkSerialization(createResponseHeader(), null); - checkSerialization(createGroupCoordinatorRequest()); - checkSerialization(createGroupCoordinatorRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createGroupCoordinatorResponse(), null); - checkSerialization(createControlledShutdownRequest()); - checkSerialization(createControlledShutdownResponse(), null); - checkSerialization(createControlledShutdownRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createFetchRequest(3), 3); - checkSerialization(createFetchRequest(3).getErrorResponse(new UnknownServerException()), 3); - checkSerialization(createFetchResponse(), null); - checkSerialization(createHeartBeatRequest()); - checkSerialization(createHeartBeatRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createHeartBeatResponse(), null); - checkSerialization(createJoinGroupRequest(1), 1); - checkSerialization(createJoinGroupRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createJoinGroupRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createJoinGroupResponse(), null); - checkSerialization(createLeaveGroupRequest()); - checkSerialization(createLeaveGroupRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createLeaveGroupResponse(), null); - checkSerialization(createListGroupsRequest()); - checkSerialization(createListGroupsRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createListGroupsResponse(), null); - checkSerialization(createDescribeGroupRequest()); - checkSerialization(createDescribeGroupRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createDescribeGroupResponse(), null); - checkSerialization(createListOffsetRequest(1), 1); - checkSerialization(createListOffsetRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createListOffsetResponse(1), 1); - checkSerialization(MetadataRequest.allTopics((short) 2), 2); - checkSerialization(createMetadataRequest(1, Arrays.asList("topic1")), 1); - checkSerialization(createMetadataRequest(1, Arrays.asList("topic1")).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createMetadataResponse(2), 2); - checkSerialization(createMetadataRequest(2, Arrays.asList("topic1")).getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createOffsetCommitRequest(2), 2); - checkSerialization(createOffsetCommitRequest(2).getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createOffsetCommitResponse(), null); - checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); - checkSerialization(OffsetFetchRequest.forAllPartitions("group1").getErrorResponse(new NotCoordinatorForGroupException()), 2); - checkSerialization(createOffsetFetchRequest(0)); - checkSerialization(createOffsetFetchRequest(1)); - checkSerialization(createOffsetFetchRequest(2)); - checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); - checkSerialization(createOffsetFetchRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createOffsetFetchRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createOffsetFetchRequest(2).getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createOffsetFetchResponse(), null); - checkSerialization(createProduceRequest()); - checkSerialization(createProduceRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createProduceResponse(), null); - checkSerialization(createStopReplicaRequest(true)); - checkSerialization(createStopReplicaRequest(false)); - checkSerialization(createStopReplicaRequest(true).getErrorResponse(new UnknownServerException()), null); - checkSerialization(createStopReplicaResponse(), null); - checkSerialization(createLeaderAndIsrRequest()); - checkSerialization(createLeaderAndIsrRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createLeaderAndIsrResponse(), null); - checkSerialization(createSaslHandshakeRequest()); - checkSerialization(createSaslHandshakeRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createSaslHandshakeResponse(), null); - checkSerialization(createApiVersionRequest()); - checkSerialization(createApiVersionRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createApiVersionResponse(), null); - checkSerialization(createCreateTopicRequest(0), 0); - checkSerialization(createCreateTopicRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createCreateTopicResponse(0), 0); - checkSerialization(createCreateTopicRequest(1), 1); - checkSerialization(createCreateTopicRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createCreateTopicResponse(1), 1); - checkSerialization(createDeleteTopicsRequest()); - checkSerialization(createDeleteTopicsRequest().getErrorResponse(new UnknownServerException()), null); - checkSerialization(createDeleteTopicsResponse(), null); + checkRequest(createGroupCoordinatorRequest()); + checkErrorResponse(createGroupCoordinatorRequest(), new UnknownServerException()); + checkResponse(createGroupCoordinatorResponse(), 0); + checkRequest(createControlledShutdownRequest()); + checkResponse(createControlledShutdownResponse(), 1); + checkErrorResponse(createControlledShutdownRequest(), new UnknownServerException()); + checkRequest(createFetchRequest(3)); + checkErrorResponse(createFetchRequest(3), new UnknownServerException()); + checkResponse(createFetchResponse(), 0); + checkRequest(createHeartBeatRequest()); + checkErrorResponse(createHeartBeatRequest(), new UnknownServerException()); + checkResponse(createHeartBeatResponse(), 0); + checkRequest(createJoinGroupRequest(1)); + checkErrorResponse(createJoinGroupRequest(0), new UnknownServerException()); + checkErrorResponse(createJoinGroupRequest(1), new UnknownServerException()); + checkResponse(createJoinGroupResponse(), 0); + checkRequest(createLeaveGroupRequest()); + checkErrorResponse(createLeaveGroupRequest(), new UnknownServerException()); + checkResponse(createLeaveGroupResponse(), 0); + checkRequest(createListGroupsRequest()); + checkErrorResponse(createListGroupsRequest(), new UnknownServerException()); + checkResponse(createListGroupsResponse(), 0); + checkRequest(createDescribeGroupRequest()); + checkErrorResponse(createDescribeGroupRequest(), new UnknownServerException()); + checkResponse(createDescribeGroupResponse(), 0); + checkRequest(createListOffsetRequest(1)); + checkErrorResponse(createListOffsetRequest(1), new UnknownServerException()); + checkResponse(createListOffsetResponse(1), 1); + checkRequest(MetadataRequest.Builder.allTopics().build((short) 2)); + checkRequest(createMetadataRequest(1, Arrays.asList("topic1"))); + checkErrorResponse(createMetadataRequest(1, Arrays.asList("topic1")), new UnknownServerException()); + checkResponse(createMetadataResponse(), 2); + checkErrorResponse(createMetadataRequest(2, Arrays.asList("topic1")), new UnknownServerException()); + checkRequest(createOffsetCommitRequest(2)); + checkErrorResponse(createOffsetCommitRequest(2), new UnknownServerException()); + checkResponse(createOffsetCommitResponse(), 0); + checkRequest(OffsetFetchRequest.forAllPartitions("group1")); + checkErrorResponse(OffsetFetchRequest.forAllPartitions("group1"), new NotCoordinatorForGroupException()); + checkRequest(createOffsetFetchRequest(0)); + checkRequest(createOffsetFetchRequest(1)); + checkRequest(createOffsetFetchRequest(2)); + checkRequest(OffsetFetchRequest.forAllPartitions("group1")); + checkErrorResponse(createOffsetFetchRequest(0), new UnknownServerException()); + checkErrorResponse(createOffsetFetchRequest(1), new UnknownServerException()); + checkErrorResponse(createOffsetFetchRequest(2), new UnknownServerException()); + checkResponse(createOffsetFetchResponse(), 0); + checkRequest(createProduceRequest()); + checkErrorResponse(createProduceRequest(), new UnknownServerException()); + checkResponse(createProduceResponse(), 2); + checkRequest(createStopReplicaRequest(true)); + checkRequest(createStopReplicaRequest(false)); + checkErrorResponse(createStopReplicaRequest(true), new UnknownServerException()); + checkResponse(createStopReplicaResponse(), 0); + checkRequest(createLeaderAndIsrRequest()); + checkErrorResponse(createLeaderAndIsrRequest(), new UnknownServerException()); + checkResponse(createLeaderAndIsrResponse(), 0); + checkRequest(createSaslHandshakeRequest()); + checkErrorResponse(createSaslHandshakeRequest(), new UnknownServerException()); + checkResponse(createSaslHandshakeResponse(), 0); + checkRequest(createApiVersionRequest()); + checkErrorResponse(createApiVersionRequest(), new UnknownServerException()); + checkResponse(createApiVersionResponse(), 0); + checkRequest(createCreateTopicRequest(0)); + checkErrorResponse(createCreateTopicRequest(0), new UnknownServerException()); + checkResponse(createCreateTopicResponse(), 0); + checkRequest(createCreateTopicRequest(1)); + checkErrorResponse(createCreateTopicRequest(1), new UnknownServerException()); + checkResponse(createCreateTopicResponse(), 1); + checkRequest(createDeleteTopicsRequest()); + checkErrorResponse(createDeleteTopicsRequest(), new UnknownServerException()); + checkResponse(createDeleteTopicsResponse(), 0); checkOlderFetchVersions(); - checkSerialization(createMetadataResponse(0), 0); - checkSerialization(createMetadataResponse(1), 1); - checkSerialization(createMetadataRequest(1, Arrays.asList("topic1")).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createOffsetCommitRequest(0), 0); - checkSerialization(createOffsetCommitRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createOffsetCommitRequest(1), 1); - checkSerialization(createOffsetCommitRequest(1).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createJoinGroupRequest(0), 0); - checkSerialization(createUpdateMetadataRequest(0, null), 0); - checkSerialization(createUpdateMetadataRequest(0, null).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createUpdateMetadataRequest(1, null), 1); - checkSerialization(createUpdateMetadataRequest(1, "rack1"), 1); - checkSerialization(createUpdateMetadataRequest(1, null).getErrorResponse(new UnknownServerException()), 1); - checkSerialization(createUpdateMetadataRequest(2, "rack1"), 2); - checkSerialization(createUpdateMetadataRequest(2, null), 2); - checkSerialization(createUpdateMetadataRequest(2, "rack1").getErrorResponse(new UnknownServerException()), 2); - checkSerialization(createUpdateMetadataRequest(3, "rack1")); - checkSerialization(createUpdateMetadataRequest(3, null)); - checkSerialization(createUpdateMetadataRequest(3, "rack1").getErrorResponse(new UnknownServerException()), 3); - checkSerialization(createUpdateMetadataResponse(), null); - checkSerialization(createListOffsetRequest(0), 0); - checkSerialization(createListOffsetRequest(0).getErrorResponse(new UnknownServerException()), 0); - checkSerialization(createListOffsetResponse(0), 0); + checkResponse(createMetadataResponse(), 0); + checkResponse(createMetadataResponse(), 1); + checkErrorResponse(createMetadataRequest(1, Arrays.asList("topic1")), new UnknownServerException()); + checkRequest(createOffsetCommitRequest(0)); + checkErrorResponse(createOffsetCommitRequest(0), new UnknownServerException()); + checkRequest(createOffsetCommitRequest(1)); + checkErrorResponse(createOffsetCommitRequest(1), new UnknownServerException()); + checkRequest(createJoinGroupRequest(0)); + checkRequest(createUpdateMetadataRequest(0, null)); + checkErrorResponse(createUpdateMetadataRequest(0, null), new UnknownServerException()); + checkRequest(createUpdateMetadataRequest(1, null)); + checkRequest(createUpdateMetadataRequest(1, "rack1")); + checkErrorResponse(createUpdateMetadataRequest(1, null), new UnknownServerException()); + checkRequest(createUpdateMetadataRequest(2, "rack1")); + checkRequest(createUpdateMetadataRequest(2, null)); + checkErrorResponse(createUpdateMetadataRequest(2, "rack1"), new UnknownServerException()); + checkRequest(createUpdateMetadataRequest(3, "rack1")); + checkRequest(createUpdateMetadataRequest(3, null)); + checkErrorResponse(createUpdateMetadataRequest(3, "rack1"), new UnknownServerException()); + checkResponse(createUpdateMetadataResponse(), 0); + checkRequest(createListOffsetRequest(0)); + checkErrorResponse(createListOffsetRequest(0), new UnknownServerException()); + checkResponse(createListOffsetResponse(0), 0); + } + + @Test + public void testRequestHeader() { + RequestHeader header = createRequestHeader(); + ByteBuffer buffer = toBuffer(header.toStruct()); + RequestHeader deserialized = RequestHeader.parse(buffer); + assertEquals(header.apiVersion(), deserialized.apiVersion()); + assertEquals(header.apiKey(), deserialized.apiKey()); + assertEquals(header.clientId(), deserialized.clientId()); + assertEquals(header.correlationId(), deserialized.correlationId()); + } + + @Test + public void testResponseHeader() { + ResponseHeader header = createResponseHeader(); + ByteBuffer buffer = toBuffer(header.toStruct()); + ResponseHeader deserialized = ResponseHeader.parse(buffer); + assertEquals(header.correlationId(), deserialized.correlationId()); } private void checkOlderFetchVersions() throws Exception { int latestVersion = ProtoUtils.latestVersion(ApiKeys.FETCH.id); for (int i = 0; i < latestVersion; ++i) { - checkSerialization(createFetchRequest(i).getErrorResponse(new UnknownServerException()), i); - checkSerialization(createFetchRequest(i), i); + checkErrorResponse(createFetchRequest(i), new UnknownServerException()); + checkRequest(createFetchRequest(i)); } } - private void checkSerialization(AbstractRequest req) throws Exception { - checkSerialization(req, Integer.valueOf(req.version())); + private void checkErrorResponse(AbstractRequest req, Throwable e) throws Exception { + checkResponse(req.getErrorResponse(e), req.version()); + } + + private void checkRequest(AbstractRequest req) throws Exception { + // Check that we can serialize, deserialize and serialize again + // We don't check for equality or hashCode because it is likely to fail for any request containing a HashMap + Struct struct = req.toStruct(); + AbstractRequest deserialized = (AbstractRequest) deserialize(req, struct, req.version()); + deserialized.toStruct(); } - private void checkSerialization(AbstractRequestResponse req, Integer version) throws Exception { - ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf()); - req.writeTo(buffer); + private void checkResponse(AbstractResponse response, int version) throws Exception { + // Check that we can serialize, deserialize and serialize again + // We don't check for equality or hashCode because it is likely to fail for any response containing a HashMap + Struct struct = response.toStruct((short) version); + AbstractResponse deserialized = (AbstractResponse) deserialize(response, struct, (short) version); + deserialized.toStruct((short) version); + } + + private AbstractRequestResponse deserialize(AbstractRequestResponse req, Struct struct, short version) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { + ByteBuffer buffer = toBuffer(struct); + Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class, Short.TYPE); + return (AbstractRequestResponse) deserializer.invoke(null, buffer, version); + } + + private ByteBuffer toBuffer(Struct struct) { + ByteBuffer buffer = ByteBuffer.allocate(struct.sizeOf()); + struct.writeTo(buffer); buffer.rewind(); - AbstractRequestResponse deserialized; - if (version == null) { - Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class); - deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer); - } else { - Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class, Integer.TYPE); - deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer, version); - } - assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + - "(version " + version + ") should be the same.", req, deserialized); - assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should have the same hashcode.", - req.hashCode(), deserialized.hashCode()); + return buffer; } @Test @@ -186,14 +215,17 @@ public void produceResponseVersionTest() { responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, 10000, Record.NO_TIMESTAMP)); ProduceResponse v0Response = new ProduceResponse(responseData); - ProduceResponse v1Response = new ProduceResponse(responseData, 10, 1); - ProduceResponse v2Response = new ProduceResponse(responseData, 10, 2); + ProduceResponse v1Response = new ProduceResponse(responseData, 10); + ProduceResponse v2Response = new ProduceResponse(responseData, 10); assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime()); assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime()); assertEquals("Throttle time must be 10", 10, v2Response.getThrottleTime()); - assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0), v0Response.toStruct().schema()); - assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 1), v1Response.toStruct().schema()); - assertEquals("Should use schema version 2", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 2), v2Response.toStruct().schema()); + assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0), + v0Response.toStruct((short) 0).schema()); + assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 1), + v1Response.toStruct((short) 1).schema()); + assertEquals("Should use schema version 2", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 2), + v2Response.toStruct((short) 2).schema()); assertEquals("Response data does not match", responseData, v0Response.responses()); assertEquals("Response data does not match", responseData, v1Response.responses()); assertEquals("Response data does not match", responseData, v2Response.responses()); @@ -206,12 +238,14 @@ public void fetchResponseVersionTest() { MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10)); responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE, 1000000, records)); - FetchResponse v0Response = new FetchResponse(0, responseData, 0); - FetchResponse v1Response = new FetchResponse(1, responseData, 10); - assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime()); - assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime()); - assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0), v0Response.toStruct().schema()); - assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 1), v1Response.toStruct().schema()); + FetchResponse v0Response = new FetchResponse(responseData, 0); + FetchResponse v1Response = new FetchResponse(responseData, 10); + assertEquals("Throttle time must be zero", 0, v0Response.throttleTimeMs()); + assertEquals("Throttle time must be 10", 10, v1Response.throttleTimeMs()); + assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0), + v0Response.toStruct((short) 0).schema()); + assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 1), + v1Response.toStruct((short) 1).schema()); assertEquals("Response data does not match", responseData, v0Response.responseData()); assertEquals("Response data does not match", responseData, v1Response.responseData()); } @@ -239,19 +273,18 @@ public void verifyFetchResponseFullWrite() throws Exception { // read the body Struct responseBody = ProtoUtils.responseSchema(ApiKeys.FETCH.id, header.apiVersion()).read(buf); - FetchResponse parsedResponse = new FetchResponse(responseBody); - assertEquals(parsedResponse, fetchResponse); + assertEquals(fetchResponse.toStruct(header.apiVersion()), responseBody); - assertEquals(size, responseHeader.sizeOf() + parsedResponse.sizeOf()); + assertEquals(size, responseHeader.sizeOf() + responseBody.sizeOf()); } @Test public void testControlledShutdownResponse() { ControlledShutdownResponse response = createControlledShutdownResponse(); - ByteBuffer buffer = ByteBuffer.allocate(response.sizeOf()); - response.writeTo(buffer); - buffer.rewind(); - ControlledShutdownResponse deserialized = ControlledShutdownResponse.parse(buffer); + short version = ProtoUtils.latestVersion(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id); + Struct struct = response.toStruct(version); + ByteBuffer buffer = toBuffer(struct); + ControlledShutdownResponse deserialized = ControlledShutdownResponse.parse(buffer, version); assertEquals(response.error(), deserialized.error()); assertEquals(response.partitionsRemaining(), deserialized.partitionsRemaining()); } @@ -259,9 +292,8 @@ public void testControlledShutdownResponse() { @Test public void testRequestHeaderWithNullClientId() { RequestHeader header = new RequestHeader((short) 10, (short) 1, null, 10); - ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf()); - header.writeTo(buffer); - buffer.rewind(); + Struct headerStruct = header.toStruct(); + ByteBuffer buffer = toBuffer(headerStruct); RequestHeader deserialized = RequestHeader.parse(buffer); assertEquals(header.apiKey(), deserialized.apiKey()); assertEquals(header.apiVersion(), deserialized.apiVersion()); @@ -294,8 +326,7 @@ private FetchRequest createFetchRequest(int version) { LinkedHashMap fetchData = new LinkedHashMap<>(); fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000)); fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000)); - return new FetchRequest.Builder(100, 100000, fetchData).setMaxBytes(1000). - setVersion((short) version).build(); + return FetchRequest.Builder.forConsumer(100, 100000, fetchData).setMaxBytes(1000).build((short) version); } private FetchResponse createFetchResponse() { @@ -320,7 +351,7 @@ private JoinGroupRequest createJoinGroupRequest(int version) { protocols.add(new JoinGroupRequest.ProtocolMetadata("consumer-range", metadata)); if (version == 0) { return new JoinGroupRequest.Builder("group1", 30000, "consumer1", "consumer", protocols). - setVersion((short) version).build(); + build((short) version); } else { return new JoinGroupRequest.Builder("group1", 10000, "consumer1", "consumer", protocols). setRebalanceTimeout(60000).build(); @@ -372,11 +403,11 @@ private ListOffsetRequest createListOffsetRequest(int version) { Map offsetData = Collections.singletonMap( new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10)); - return new ListOffsetRequest.Builder().setOffsetData(offsetData).setVersion((short) version).build(); + return ListOffsetRequest.Builder.forConsumer((short) 0).setOffsetData(offsetData).build((short) version); } else if (version == 1) { Map offsetData = Collections.singletonMap( new TopicPartition("test", 0), 1000000L); - return new ListOffsetRequest.Builder().setTargetTimes(offsetData).setVersion((short) version).build(); + return ListOffsetRequest.Builder.forConsumer((short) 1).setTargetTimes(offsetData).build((short) version); } else { throw new IllegalArgumentException("Illegal ListOffsetRequest version " + version); } @@ -386,24 +417,24 @@ private ListOffsetRequest createListOffsetRequest(int version) { private ListOffsetResponse createListOffsetResponse(int version) { if (version == 0) { Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE, Arrays.asList(100L))); + responseData.put(new TopicPartition("test", 0), + new ListOffsetResponse.PartitionData(Errors.NONE, Arrays.asList(100L))); return new ListOffsetResponse(responseData); } else if (version == 1) { Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE, 10000L, 100L)); - return new ListOffsetResponse(responseData, 1); + responseData.put(new TopicPartition("test", 0), + new ListOffsetResponse.PartitionData(Errors.NONE, 10000L, 100L)); + return new ListOffsetResponse(responseData); } else { throw new IllegalArgumentException("Illegal ListOffsetResponse version " + version); } } private MetadataRequest createMetadataRequest(int version, List topics) { - return new MetadataRequest.Builder(topics). - setVersion((short) version). - build(); + return new MetadataRequest.Builder(topics).build((short) version); } - private MetadataResponse createMetadataResponse(int version) { + private MetadataResponse createMetadataResponse() { Node node = new Node(1, "host1", 1001); List replicas = Arrays.asList(node); List isr = Arrays.asList(node); @@ -414,7 +445,7 @@ private MetadataResponse createMetadataResponse(int version) { allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", false, Collections.emptyList())); - return new MetadataResponse(Arrays.asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata, version); + return new MetadataResponse(Arrays.asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata); } private OffsetCommitRequest createOffsetCommitRequest(int version) { @@ -425,8 +456,7 @@ private OffsetCommitRequest createOffsetCommitRequest(int version) { .setGenerationId(100) .setMemberId("consumer1") .setRetentionTime(1000000) - .setVersion((short) version) - .build(); + .build((short) version); } private OffsetCommitResponse createOffsetCommitResponse() { @@ -437,8 +467,7 @@ private OffsetCommitResponse createOffsetCommitResponse() { private OffsetFetchRequest createOffsetFetchRequest(int version) { return new OffsetFetchRequest.Builder("group1", singletonList(new TopicPartition("test11", 1))) - .setVersion((short) version) - .build(); + .build((short) version); } private OffsetFetchResponse createOffsetFetchResponse() { @@ -540,8 +569,8 @@ private UpdateMetadataRequest createUpdateMetadataRequest(int version, String ra new UpdateMetadataRequest.Broker(0, endPoints1, rack), new UpdateMetadataRequest.Broker(1, endPoints2, rack) )); - return new UpdateMetadataRequest.Builder(1, 10, partitionStates, liveBrokers). - setVersion((short) version).build(); + return new UpdateMetadataRequest.Builder((short) version, 1, 10, partitionStates, + liveBrokers).build(); } private UpdateMetadataResponse createUpdateMetadataResponse() { @@ -584,14 +613,14 @@ private CreateTopicsRequest createCreateTopicRequest(int version, boolean valida Map request = new HashMap<>(); request.put("my_t1", request1); request.put("my_t2", request2); - return new CreateTopicsRequest.Builder(request, 0, validateOnly).setVersion((short) version).build(); + return new CreateTopicsRequest.Builder(request, 0, validateOnly).build((short) version); } - private CreateTopicsResponse createCreateTopicResponse(int version) { + private CreateTopicsResponse createCreateTopicResponse() { Map errors = new HashMap<>(); errors.put("t1", new CreateTopicsResponse.Error(Errors.INVALID_TOPIC_EXCEPTION, null)); errors.put("t2", new CreateTopicsResponse.Error(Errors.LEADER_NOT_AVAILABLE, "Leader with id 5 is not available.")); - return new CreateTopicsResponse(errors, (short) version); + return new CreateTopicsResponse(errors); } private DeleteTopicsRequest createDeleteTopicsRequest() { diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 76fb9b31836cc..3a9e0cec0d646 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -392,7 +392,7 @@ public void testApiVersionsRequestWithUnsupportedVersion() throws Exception { selector.send(request.toSend(node, header)); ByteBuffer responseBuffer = waitForResponse(); ResponseHeader.parse(responseBuffer); - ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer); + ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, (short) 0); assertEquals(Errors.UNSUPPORTED_VERSION, response.error()); // Send ApiVersionsRequest with a supported version. This should succeed. diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 97da1f5d84a7c..f0498212eb855 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -205,7 +205,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, responseData.put(new TopicPartition(topic, partition), new JFetchResponse.PartitionData(Errors.forException(e), -1, MemoryRecords.EMPTY)) } - val errorResponse = new JFetchResponse(versionId, responseData, 0) + val errorResponse = new JFetchResponse(responseData, 0) // Magic value does not matter here because the message set is empty requestChannel.sendResponse(new RequestChannel.Response(request, errorResponse)) } diff --git a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala deleted file mode 100644 index 3783c29ccbe70..0000000000000 --- a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala +++ /dev/null @@ -1,56 +0,0 @@ -/** - * 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.nio.ByteBuffer - -import kafka.api.ApiUtils._ -import org.apache.kafka.common.requests.AbstractResponse - -private[kafka] abstract class GenericRequestAndHeader(val versionId: Short, - val correlationId: Int, - val clientId: String, - val body: AbstractResponse, - val name: String, - override val requestId: Option[Short] = None) - extends RequestOrResponse(requestId) { - - def writeTo(buffer: ByteBuffer) { - buffer.putShort(versionId) - buffer.putInt(correlationId) - writeShortString(buffer, clientId) - body.writeTo(buffer) - } - - def sizeInBytes(): Int = { - 2 /* version id */ + - 4 /* correlation id */ + - (2 + clientId.length) /* client id */ + - body.sizeOf() - } - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val strBuffer = new StringBuilder - strBuffer.append("Name: " + name) - strBuffer.append("; Version: " + versionId) - strBuffer.append("; CorrelationId: " + correlationId) - strBuffer.append("; ClientId: " + clientId) - strBuffer.append("; Body: " + body.toString) - strBuffer.toString() - } -} diff --git a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala deleted file mode 100644 index be0c080fbe15e..0000000000000 --- a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala +++ /dev/null @@ -1,47 +0,0 @@ -/** - * 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.nio.ByteBuffer - -import org.apache.kafka.common.requests.AbstractResponse - -private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int, - val body: AbstractResponse, - val name: String, - override val requestId: Option[Short] = None) - extends RequestOrResponse(requestId) { - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(correlationId) - body.writeTo(buffer) - } - - def sizeInBytes(): Int = { - 4 /* correlation id */ + - body.sizeOf() - } - - override def toString: String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val strBuffer = new StringBuilder - strBuffer.append("Name: " + name) - strBuffer.append("; CorrelationId: " + correlationId) - strBuffer.append("; Body: " + body.toString) - strBuffer.toString() - } -} diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index a87cdc90b3fed..bd483887bc47f 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -129,7 +129,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, } override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - if(request.body.asInstanceOf[org.apache.kafka.common.requests.ProduceRequest].acks == 0) { + if (request.body[org.apache.kafka.common.requests.ProduceRequest].acks == 0) { requestChannel.closeConnection(request.processor, request) } else { diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index d8e6a9504d6f6..a2fee6b6b4e1c 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -364,9 +364,9 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging partitionStateInfo.allReplicas.map(Integer.valueOf).asJava) topicPartition -> partitionState } - val leaderAndIsrRequest = new LeaderAndIsrRequest. - Builder(controllerId, controllerEpoch, partitionStates.asJava, leaders.asJava) - controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest, null) + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(controllerId, controllerEpoch, partitionStates.asJava, + leaders.asJava) + controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest) } leaderAndIsrRequestMap.clear() @@ -405,9 +405,8 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull) } } - new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, liveBrokers.asJava). - setVersion(version) + new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava, + liveBrokers.asJava) } updateMetadataRequestBrokerSet.foreach { broker => diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index c063801edfc09..a5d71602eaeef 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -19,7 +19,7 @@ package kafka.network import java.net.InetAddress import java.nio.ByteBuffer -import java.util.HashMap +import java.util.Collections import java.util.concurrent._ import com.yammer.metrics.core.Gauge @@ -37,16 +37,18 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Time import org.apache.log4j.Logger +import scala.reflect.{classTag, ClassTag} + object RequestChannel extends Logging { val AllDone = Request(processor = 1, connectionId = "2", Session(KafkaPrincipal.ANONYMOUS, InetAddress.getLocalHost), - buffer = getShutdownReceive, startTimeMs = 0, listenerName = new ListenerName(""), + buffer = shutdownReceive, startTimeMs = 0, listenerName = new ListenerName(""), securityProtocol = SecurityProtocol.PLAINTEXT) private val requestLogger = Logger.getLogger("kafka.request.logger") - private def getShutdownReceive = { - val emptyProduceRequest = new ProduceRequest.Builder(0, 0, new HashMap[TopicPartition, MemoryRecords]()).build() + private def shutdownReceive: ByteBuffer = { + val emptyProduceRequest = new ProduceRequest.Builder(0, 0, Collections.emptyMap[TopicPartition, MemoryRecords]).build() val emptyRequestHeader = new RequestHeader(ApiKeys.PRODUCE.id, emptyProduceRequest.version, "", 0) - AbstractRequestResponse.serialize(emptyRequestHeader, emptyProduceRequest) + emptyProduceRequest.serialize(emptyRequestHeader) } case class Session(principal: KafkaPrincipal, clientAddress: InetAddress) { @@ -84,12 +86,13 @@ object RequestChannel extends Logging { } } else null - val body: AbstractRequest = + val bodyAndSize: RequestAndSize = if (requestObj == null) try { // For unsupported version of ApiVersionsRequest, create a dummy request to enable an error response to be returned later - if (header.apiKey == ApiKeys.API_VERSIONS.id && !Protocol.apiVersionSupported(header.apiKey, header.apiVersion)) - new ApiVersionsRequest.Builder().build() + if (header.apiKey == ApiKeys.API_VERSIONS.id && !Protocol.apiVersionSupported(header.apiKey, header.apiVersion)) { + new RequestAndSize(new ApiVersionsRequest.Builder().build(), 0) + } else AbstractRequest.getRequest(header.apiKey, header.apiVersion, buffer) } catch { @@ -108,6 +111,14 @@ object RequestChannel extends Logging { s"$header -- $body" } + def body[T <: AbstractRequest : ClassTag] = { + bodyAndSize.request match { + case r: T => r + case r => + throw new ClassCastException(s"Expected request with type ${classTag[T].runtimeClass}, but found ${r.getClass}") + } + } + trace("Processor %d received request : %s".format(processor, requestDesc(true))) def updateRequestMetrics() { @@ -132,7 +143,7 @@ object RequestChannel extends Logging { val totalTime = endTimeMs - startTimeMs val fetchMetricNames = if (requestId == ApiKeys.FETCH.id) { - val isFromFollower = body.asInstanceOf[FetchRequest].isFromFollower + val isFromFollower = body[FetchRequest].isFromFollower Seq( if (isFromFollower) RequestMetrics.followFetchMetricName else RequestMetrics.consumerFetchMetricName @@ -163,11 +174,8 @@ object RequestChannel extends Logging { case class Response(processor: Int, request: Request, responseSend: Send, responseAction: ResponseAction) { request.responseCompleteTimeMs = Time.SYSTEM.milliseconds - def this(processor: Int, request: Request, responseSend: Send) = - this(processor, request, responseSend, if (responseSend == null) NoOpAction else SendAction) - - def this(request: Request, send: Send) = - this(request.processor, request, send) + def this(request: Request, responseSend: Send) = + this(request.processor, request, responseSend, if (responseSend == null) NoOpAction else SendAction) def this(request: Request, response: AbstractResponse) = this(request, response.toSend(request.connectionId, request.header)) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 130821681ac6c..c90cace9bcfcb 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -108,7 +108,7 @@ class KafkaApis(val requestChannel: RequestChannel, request.requestObj.handleError(e, requestChannel, request) error("Error when handling request %s".format(request.requestObj), e) } else { - val response = request.body.getErrorResponse(e) + val response = request.body[AbstractRequest].getErrorResponse(e) /* If request doesn't have a default error response, we just close the connection. For example, when produce request has acks set to 0 */ @@ -128,7 +128,7 @@ class KafkaApis(val requestChannel: RequestChannel, // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted val correlationId = request.header.correlationId - val leaderAndIsrRequest = request.body.asInstanceOf[LeaderAndIsrRequest] + val leaderAndIsrRequest = request.body[LeaderAndIsrRequest] try { def onLeadershipChange(updatedLeaders: Iterable[Partition], updatedFollowers: Iterable[Partition]) { @@ -167,7 +167,7 @@ class KafkaApis(val requestChannel: RequestChannel, // ensureTopicExists is only for client facing requests // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted - val stopReplicaRequest = request.body.asInstanceOf[StopReplicaRequest] + val stopReplicaRequest = request.body[StopReplicaRequest] val response = if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { @@ -195,7 +195,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleUpdateMetadataRequest(request: RequestChannel.Request) { val correlationId = request.header.correlationId - val updateMetadataRequest = request.body.asInstanceOf[UpdateMetadataRequest] + val updateMetadataRequest = request.body[UpdateMetadataRequest] val updateMetadataResponse = if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { @@ -235,7 +235,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetCommitRequest(request: RequestChannel.Request) { val header = request.header - val offsetCommitRequest = request.body.asInstanceOf[OffsetCommitRequest] + val offsetCommitRequest = request.body[OffsetCommitRequest] // reject the request if not authorized to the group if (!authorize(request.session, Read, new Resource(Group, offsetCommitRequest.groupId))) { @@ -247,14 +247,13 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, response)) } else { val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = offsetCommitRequest.offsetData.asScala.toMap.partition { - case (topicPartition, _) => { + case (topicPartition, _) => val authorizedForDescribe = authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) val exists = metadataCache.contains(topicPartition.topic) if (!authorizedForDescribe && exists) debug(s"Offset commit request with correlation id ${header.correlationId} from client ${header.clientId} " + s"on partition $topicPartition failing due to user not having DESCRIBE authorization, but returning UNKNOWN_TOPIC_OR_PARTITION") authorizedForDescribe && exists - } } val (authorizedTopics, unauthorizedForReadTopics) = existingAndAuthorizedForDescribeTopics.partition { @@ -349,8 +348,8 @@ class KafkaApis(val requestChannel: RequestChannel, * Handle a produce request */ def handleProducerRequest(request: RequestChannel.Request) { - val produceRequest = request.body.asInstanceOf[ProduceRequest] - val numBytesAppended = request.header.sizeOf + produceRequest.sizeOf + val produceRequest = request.body[ProduceRequest] + val numBytesAppended = request.header.toStruct.sizeOf + request.bodyAndSize.size val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = produceRequest.partitionRecords.asScala.partition { case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) && metadataCache.contains(topicPartition.topic) @@ -399,14 +398,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.noOperation(request.processor, request) } } else { - val respBody = request.header.apiVersion match { - case 0 => new ProduceResponse(mergedResponseStatus.asJava) - case version@(1 | 2) => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, version) - // This case shouldn't happen unless a new version of ProducerRequest is added without - // updating this part of the code to handle it properly. - case version => throw new IllegalArgumentException(s"Version `$version` of ProduceRequest is not handled. Code must be updated.") - } - + val respBody = new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, respBody)) } } @@ -445,7 +437,7 @@ class KafkaApis(val requestChannel: RequestChannel, * Handle a fetch request */ def handleFetchRequest(request: RequestChannel.Request) { - val fetchRequest = request.body.asInstanceOf[FetchRequest] + val fetchRequest = request.body[FetchRequest] val versionId = request.header.apiVersion val clientId = request.header.clientId @@ -505,13 +497,13 @@ class KafkaApis(val requestChannel: RequestChannel, BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.records.sizeInBytes) } - val response = new FetchResponse(versionId, fetchedPartitionData, 0) + val response = new FetchResponse(fetchedPartitionData, 0) + val responseStruct = response.toStruct(versionId) - def fetchResponseCallback(delayTimeMs: Int) { - trace(s"Sending fetch response to client $clientId of " + - s"${convertedPartitionData.map { case (_, v) => v.records.sizeInBytes }.sum} bytes") - val fetchResponse = if (delayTimeMs > 0) new FetchResponse(versionId, fetchedPartitionData, delayTimeMs) else response - requestChannel.sendResponse(new RequestChannel.Response(request, fetchResponse)) + def fetchResponseCallback(throttleTimeMs: Int) { + trace(s"Sending fetch response to client $clientId of ${responseStruct.sizeOf} bytes.") + val responseSend = response.toSend(responseStruct, throttleTimeMs, request.connectionId, request.header) + requestChannel.sendResponse(new RequestChannel.Response(request, responseSend)) } // When this callback is triggered, the remote API call has completed @@ -521,9 +513,10 @@ class KafkaApis(val requestChannel: RequestChannel, // We've already evaluated against the quota and are good to go. Just need to record it now. val responseSize = sizeOfThrottledPartitions(versionId, fetchRequest, mergedPartitionData, quotas.leader) quotas.leader.record(responseSize) - fetchResponseCallback(0) + fetchResponseCallback(throttleTimeMs = 0) } else { - quotas.fetch.recordAndMaybeThrottle(request.session.sanitizedUser, clientId, response.sizeOf, fetchResponseCallback) + quotas.fetch.recordAndMaybeThrottle(request.session.sanitizedUser, clientId, responseStruct.sizeOf, + fetchResponseCallback) } } @@ -547,7 +540,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest: FetchRequest, mergedPartitionData: Seq[(TopicPartition, FetchResponse.PartitionData)], quota: ReplicationQuotaManager): Int = { - val partitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]() + val partitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData] mergedPartitionData.foreach { case (tp, data) => if (quota.isThrottled(tp)) partitionData.put(tp, data) @@ -570,14 +563,14 @@ class KafkaApis(val requestChannel: RequestChannel, else handleListOffsetRequestV1(request) - val response = new ListOffsetResponse(mergedResponseMap.asJava, version) + val response = new ListOffsetResponse(mergedResponseMap.asJava) requestChannel.sendResponse(new RequestChannel.Response(request, response)) } private def handleListOffsetRequestV0(request : RequestChannel.Request) : Map[TopicPartition, ListOffsetResponse.PartitionData] = { val correlationId = request.header.correlationId val clientId = request.header.clientId - val offsetRequest = request.body.asInstanceOf[ListOffsetRequest] + val offsetRequest = request.body[ListOffsetRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.offsetData.asScala.partition { case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) @@ -628,7 +621,7 @@ class KafkaApis(val requestChannel: RequestChannel, private def handleListOffsetRequestV1(request : RequestChannel.Request): Map[TopicPartition, ListOffsetResponse.PartitionData] = { val correlationId = request.header.correlationId val clientId = request.header.clientId - val offsetRequest = request.body.asInstanceOf[ListOffsetRequest] + val offsetRequest = request.body[ListOffsetRequest] val (authorizedRequestInfo, unauthorizedRequestInfo) = offsetRequest.partitionTimestamps.asScala.partition { case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) @@ -824,7 +817,7 @@ class KafkaApis(val requestChannel: RequestChannel, * Handle a topic metadata request */ def handleTopicMetadataRequest(request: RequestChannel.Request) { - val metadataRequest = request.body.asInstanceOf[MetadataRequest] + val metadataRequest = request.body[MetadataRequest] val requestVersion = request.header.apiVersion() val topics = @@ -889,8 +882,7 @@ class KafkaApis(val requestChannel: RequestChannel, brokers.map(_.getNode(request.listenerName)).asJava, clusterId, metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID), - completeTopicMetadata.asJava, - requestVersion + completeTopicMetadata.asJava ) requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) } @@ -900,7 +892,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val header = request.header - val offsetFetchRequest = request.body.asInstanceOf[OffsetFetchRequest] + val offsetFetchRequest = request.body[OffsetFetchRequest] def authorizeTopicDescribe(partition: TopicPartition) = authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) @@ -938,7 +930,7 @@ class KafkaApis(val requestChannel: RequestChannel, }.toMap val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap - new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava) } else { // versions 1 and above read offsets from Kafka if (offsetFetchRequest.isAllPartitions) { @@ -948,7 +940,7 @@ class KafkaApis(val requestChannel: RequestChannel, else { // clients are not allowed to see offsets for topics that are not authorized for Describe val authorizedPartitionData = allPartitionData.filter { case (topicPartition, _) => authorizeTopicDescribe(topicPartition) } - new OffsetFetchResponse(Errors.NONE, authorizedPartitionData.asJava, header.apiVersion) + new OffsetFetchResponse(Errors.NONE, authorizedPartitionData.asJava) } } else { val (authorizedPartitions, unauthorizedPartitions) = offsetFetchRequest.partitions.asScala @@ -959,7 +951,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetFetchRequest.getErrorResponse(error) else { val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap - new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava) } } } @@ -970,7 +962,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleGroupCoordinatorRequest(request: RequestChannel.Request) { - val groupCoordinatorRequest = request.body.asInstanceOf[GroupCoordinatorRequest] + val groupCoordinatorRequest = request.body[GroupCoordinatorRequest] if (!authorize(request.session, Describe, new Resource(Group, groupCoordinatorRequest.groupId))) { val responseBody = new GroupCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode) @@ -1003,7 +995,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDescribeGroupRequest(request: RequestChannel.Request) { - val describeRequest = request.body.asInstanceOf[DescribeGroupsRequest] + val describeRequest = request.body[DescribeGroupsRequest] val groups = describeRequest.groupIds().asScala.map { groupId => if (!authorize(request.session, Describe, new Resource(Group, groupId))) { @@ -1036,12 +1028,12 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleJoinGroupRequest(request: RequestChannel.Request) { - val joinGroupRequest = request.body.asInstanceOf[JoinGroupRequest] + val joinGroupRequest = request.body[JoinGroupRequest] // the callback for sending a join-group response def sendResponseCallback(joinResult: JoinGroupResult) { val members = joinResult.members map { case (memberId, metadataArray) => (memberId, ByteBuffer.wrap(metadataArray)) } - val responseBody = new JoinGroupResponse(request.header.apiVersion, joinResult.error, joinResult.generationId, + val responseBody = new JoinGroupResponse(joinResult.error, joinResult.generationId, joinResult.subProtocol, joinResult.memberId, joinResult.leaderId, members.asJava) trace("Sending join group response %s for correlation id %d to client %s." @@ -1051,7 +1043,6 @@ class KafkaApis(val requestChannel: RequestChannel, if (!authorize(request.session, Read, new Resource(Group, joinGroupRequest.groupId()))) { val responseBody = new JoinGroupResponse( - request.header.apiVersion, Errors.GROUP_AUTHORIZATION_FAILED, JoinGroupResponse.UNKNOWN_GENERATION_ID, JoinGroupResponse.UNKNOWN_PROTOCOL, @@ -1077,7 +1068,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleSyncGroupRequest(request: RequestChannel.Request) { - val syncGroupRequest = request.body.asInstanceOf[SyncGroupRequest] + val syncGroupRequest = request.body[SyncGroupRequest] def sendResponseCallback(memberState: Array[Byte], error: Errors) { val responseBody = new SyncGroupResponse(error, ByteBuffer.wrap(memberState)) @@ -1098,7 +1089,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleHeartbeatRequest(request: RequestChannel.Request) { - val heartbeatRequest = request.body.asInstanceOf[HeartbeatRequest] + val heartbeatRequest = request.body[HeartbeatRequest] // the callback for sending a heartbeat response def sendResponseCallback(error: Errors) { @@ -1123,7 +1114,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleLeaveGroupRequest(request: RequestChannel.Request) { - val leaveGroupRequest = request.body.asInstanceOf[LeaveGroupRequest] + val leaveGroupRequest = request.body[LeaveGroupRequest] // the callback for sending a leave-group response def sendResponseCallback(error: Errors) { @@ -1157,11 +1148,11 @@ class KafkaApis(val requestChannel: RequestChannel, // If this is considered to leak information about the broker version a workaround is to use SSL // with client authentication which is performed at an earlier stage of the connection where the // ApiVersionRequest is not available. - val responseBody = if (Protocol.apiVersionSupported(ApiKeys.API_VERSIONS.id, request.header.apiVersion)) - ApiVersionsResponse.API_VERSIONS_RESPONSE - else - ApiVersionsResponse.fromError(Errors.UNSUPPORTED_VERSION) - requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) + val responseSend = + if (Protocol.apiVersionSupported(ApiKeys.API_VERSIONS.id, request.header.apiVersion)) + ApiVersionsResponse.API_VERSIONS_RESPONSE.toSend(request.connectionId, request.header) + else ApiVersionsResponse.unsupportedVersionSend(request.connectionId, request.header) + requestChannel.sendResponse(new RequestChannel.Response(request, responseSend)) } def close() { @@ -1170,10 +1161,10 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleCreateTopicsRequest(request: RequestChannel.Request) { - val createTopicsRequest = request.body.asInstanceOf[CreateTopicsRequest] + val createTopicsRequest = request.body[CreateTopicsRequest] def sendResponseCallback(results: Map[String, CreateTopicsResponse.Error]): Unit = { - val responseBody = new CreateTopicsResponse(results.asJava, request.header.apiVersion) + val responseBody = new CreateTopicsResponse(results.asJava) trace(s"Sending create topics response $responseBody for correlation id ${request.header.correlationId} to client ${request.header.clientId}.") requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) } @@ -1220,7 +1211,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDeleteTopicsRequest(request: RequestChannel.Request) { - val deleteTopicRequest = request.body.asInstanceOf[DeleteTopicsRequest] + val deleteTopicRequest = request.body[DeleteTopicsRequest] val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = deleteTopicRequest.topics.asScala.partition { topic => authorize(request.session, Describe, new Resource(auth.Topic, topic)) && metadataCache.contains(topic) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index c99d7c5fe2011..7fb02a30b549a 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -261,17 +261,13 @@ class ReplicaFetcherThread(name: String, } - private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long, consumerId: Int): Long = { + private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long, replicaId: Int): Long = { val requestBuilder = if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) { val partitions = Map(topicPartition -> (earliestOrLatest: java.lang.Long)) - new ListOffsetRequest.Builder(consumerId). - setTargetTimes(partitions.asJava). - setVersion(1) + ListOffsetRequest.Builder.forReplica(1, replicaId).setTargetTimes(partitions.asJava) } else { val partitions = Map(topicPartition -> new ListOffsetRequest.PartitionData(earliestOrLatest, 1)) - new ListOffsetRequest.Builder(consumerId). - setOffsetData(partitions.asJava). - setVersion(0) + ListOffsetRequest.Builder.forReplica(0, replicaId).setOffsetData(partitions.asJava) } val clientResponse = sendRequest(requestBuilder) val response = clientResponse.responseBody.asInstanceOf[ListOffsetResponse] @@ -295,9 +291,8 @@ class ReplicaFetcherThread(name: String, requestMap.put(topicPartition, new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize)) } - val requestBuilder = new JFetchRequest.Builder(maxWait, minBytes, requestMap). - setReplicaId(replicaId).setMaxBytes(maxBytes) - requestBuilder.setVersion(fetchRequestVersion) + val requestBuilder = JFetchRequest.Builder.forReplica(fetchRequestVersion, replicaId, maxWait, minBytes, requestMap) + .setMaxBytes(maxBytes) new FetchRequest(requestBuilder) } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 3285bf2ca560f..4f71258219d3f 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.errors._ -import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} import org.apache.kafka.common.requests._ import CreateTopicsRequest.TopicDetails import org.apache.kafka.common.security.auth.KafkaPrincipal @@ -191,11 +191,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def createFetchRequest = { val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData] partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 100)) - new requests.FetchRequest.Builder(100, Int.MaxValue, partitionMap).setReplicaId(5000).build() + val version = ProtoUtils.latestVersion(ApiKeys.FETCH.id) + requests.FetchRequest.Builder.forReplica(version, 5000, 100, Int.MaxValue, partitionMap).build() } private def createListOffsetsRequest = { - new requests.ListOffsetRequest.Builder().setTargetTimes( + requests.ListOffsetRequest.Builder.forConsumer(0).setTargetTimes( Map(tp -> (0L: java.lang.Long)).asJava). build() } @@ -214,7 +215,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId, Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava - new requests.UpdateMetadataRequest.Builder(brokerId, Int.MaxValue, partitionState, brokers).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, partitionState, brokers).build() } private def createJoinGroupRequest = { @@ -770,17 +772,18 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testUnauthorizedDeleteWithoutDescribe() { - val response = send(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val deleteResponse = DeleteTopicsResponse.parse(response) - + val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) + val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, deleteResponse.errors.asScala.head._2) } @Test def testUnauthorizedDeleteWithDescribe() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), deleteTopicResource) - val response = send(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val deleteResponse = DeleteTopicsResponse.parse(response) + val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) + val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, deleteResponse.errors.asScala.head._2) } @@ -788,8 +791,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testDeleteWithWildCardAuth() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Delete)), new Resource(Topic, "*")) - val response = send(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val deleteResponse = DeleteTopicsResponse.parse(response) + val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) + val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.NONE, deleteResponse.errors.asScala.head._2) } @@ -807,8 +811,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { isAuthorized: Boolean, isAuthorizedTopicDescribe: Boolean, topicExists: Boolean = true): AbstractResponse = { - val resp = send(request, apiKey) - val response = RequestKeyToResponseDeserializer(apiKey).getMethod("parse", classOf[ByteBuffer]).invoke(null, resp).asInstanceOf[AbstractResponse] + val resp = connectAndSend(request, apiKey) + val response = RequestKeyToResponseDeserializer(apiKey).getMethod("parse", classOf[ByteBuffer], classOf[Short]).invoke( + null, resp, request.version: java.lang.Short).asInstanceOf[AbstractResponse] val error = RequestKeyToError(apiKey).asInstanceOf[(AbstractResponse) => Errors](response) val authorizationErrorCodes = resources.flatMap { resourceType => @@ -877,7 +882,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def sendOffsetFetchRequest(request: requests.OffsetFetchRequest, socketServer: SocketServer): requests.OffsetFetchResponse = { - val response = send(request, ApiKeys.OFFSET_FETCH, socketServer) + val response = connectAndSend(request, ApiKeys.OFFSET_FETCH, socketServer) requests.OffsetFetchResponse.parse(response, request.version) } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 37bc23875fcfe..387560465af2b 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, NetworkSend} import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} import org.apache.kafka.common.record.MemoryRecords -import org.apache.kafka.common.requests.{ProduceRequest, RequestHeader} +import org.apache.kafka.common.requests.{AbstractRequest, ProduceRequest, RequestHeader} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Time import org.junit.Assert._ @@ -89,13 +89,11 @@ class SocketServerTest extends JUnitSuite { } def processRequest(channel: RequestChannel, request: RequestChannel.Request) { - val byteBuffer = ByteBuffer.allocate(request.header.sizeOf + request.body.sizeOf) - request.header.writeTo(byteBuffer) - request.body.writeTo(byteBuffer) + val byteBuffer = request.body[AbstractRequest].serialize(request.header) byteBuffer.rewind() val send = new NetworkSend(request.connectionId, byteBuffer) - channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) + channel.sendResponse(new RequestChannel.Response(request, send)) } def connect(s: SocketServer = server, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { @@ -119,14 +117,11 @@ class SocketServerTest extends JUnitSuite { val ackTimeoutMs = 10000 val ack = 0: Short - val emptyRequest = new ProduceRequest.Builder( - ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() + val emptyRequest = new ProduceRequest.Builder(ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() val emptyHeader = new RequestHeader(apiKey, emptyRequest.version, clientId, correlationId) - - val byteBuffer = ByteBuffer.allocate(emptyHeader.sizeOf + emptyRequest.sizeOf) - emptyHeader.writeTo(byteBuffer) - emptyRequest.writeTo(byteBuffer) + val byteBuffer = emptyRequest.serialize(emptyHeader) byteBuffer.rewind() + val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) serializedBytes @@ -289,13 +284,10 @@ class SocketServerTest extends JUnitSuite { val clientId = "" val ackTimeoutMs = 10000 val ack = 0: Short - val emptyRequest = new ProduceRequest.Builder( - ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() + val emptyRequest = new ProduceRequest.Builder(ack, ackTimeoutMs, new HashMap[TopicPartition, MemoryRecords]()).build() val emptyHeader = new RequestHeader(apiKey, emptyRequest.version, clientId, correlationId) - val byteBuffer = ByteBuffer.allocate(emptyHeader.sizeOf() + emptyRequest.sizeOf()) - emptyHeader.writeTo(byteBuffer) - emptyRequest.writeTo(byteBuffer) + val byteBuffer = emptyRequest.serialize(emptyHeader) byteBuffer.rewind() val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) @@ -355,7 +347,7 @@ class SocketServerTest extends JUnitSuite { // detected. If the buffer is larger than 102400 bytes, a second write is attempted and it fails with an // IOException. val send = new NetworkSend(request.connectionId, ByteBuffer.allocate(550000)) - channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) + channel.sendResponse(new RequestChannel.Response(request, send)) TestUtils.waitUntilTrue(() => totalTimeHistCount() == expectedTotalTimeCount, s"request metrics not updated, expected: $expectedTotalTimeCount, actual: ${totalTimeHistCount()}") diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index ef985311f271c..448fce1a54c2e 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -82,13 +82,13 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { protected def error(error: Errors, errorMessage: Option[String] = None): CreateTopicsResponse.Error = new CreateTopicsResponse.Error(error, errorMessage.orNull) - protected def duplicateFirstTopic(request: CreateTopicsRequest) = { + protected def toStructWithDuplicateFirstTopic(request: CreateTopicsRequest): Struct = { val struct = request.toStruct val topics = struct.getArray("create_topic_requests") val firstTopic = topics(0).asInstanceOf[Struct] val newTopics = firstTopic :: topics.toList struct.set("create_topic_requests", newTopics.toArray) - new CreateTopicsRequest(struct, request.version) + struct } protected def addPartitionsAndReplicationFactorToFirstTopic(request: CreateTopicsRequest) = { @@ -102,8 +102,10 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { protected def validateErrorCreateTopicsRequests(request: CreateTopicsRequest, expectedResponse: Map[String, CreateTopicsResponse.Error], - checkErrorMessage: Boolean = true): Unit = { - val response = sendCreateTopicRequest(request) + checkErrorMessage: Boolean = true, + requestStruct: Option[Struct] = None): Unit = { + val response = requestStruct.map(sendCreateTopicRequestStruct(_, request.version)).getOrElse( + sendCreateTopicRequest(request)) val errors = response.errors.asScala assertEquals("The response size should match", expectedResponse.size, response.errors.size) @@ -133,14 +135,20 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { assignments.map { case (k, v) => (k: Integer, v.map { i => i: Integer }.asJava) }.asJava } + protected def sendCreateTopicRequestStruct(requestStruct: Struct, apiVersion: Short, + socketServer: SocketServer = controllerSocketServer): CreateTopicsResponse = { + val response = connectAndSendStruct(requestStruct, ApiKeys.CREATE_TOPICS, apiVersion, socketServer) + CreateTopicsResponse.parse(response, apiVersion) + } + protected def sendCreateTopicRequest(request: CreateTopicsRequest, socketServer: SocketServer = controllerSocketServer): CreateTopicsResponse = { - val response = send(request, ApiKeys.CREATE_TOPICS, socketServer) + val response = connectAndSend(request, ApiKeys.CREATE_TOPICS, socketServer) CreateTopicsResponse.parse(response, request.version) } protected def sendMetadataRequest(request: MetadataRequest, destination: SocketServer = anySocketServer): MetadataResponse = { val version = ProtoUtils.latestVersion(ApiKeys.METADATA.id) - val response = send(request, ApiKeys.METADATA, destination = destination) + val response = connectAndSend(request, ApiKeys.METADATA, destination = destination) MetadataResponse.parse(response, version) } diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index ffe82d162e79d..248b91e9f211c 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -17,8 +17,7 @@ package kafka.server -import org.apache.kafka.common.protocol.types.Struct -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils} +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} import org.junit.Assert._ @@ -51,14 +50,13 @@ class ApiVersionsRequestTest extends BaseRequestTest { @Test def testApiVersionsRequestWithUnsupportedVersion() { - val apiVersionsRequest = new ApiVersionsRequest( - new Struct(ProtoUtils.currentRequestSchema(ApiKeys.API_VERSIONS.id)), Short.MaxValue) - val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest) + val apiVersionsRequest = new ApiVersionsRequest(0) + val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest, Some(Short.MaxValue)) assertEquals(Errors.UNSUPPORTED_VERSION, apiVersionsResponse.error) } - private def sendApiVersionsRequest(request: ApiVersionsRequest): ApiVersionsResponse = { - val response = send(request, ApiKeys.API_VERSIONS) - ApiVersionsResponse.parse(response) + private def sendApiVersionsRequest(request: ApiVersionsRequest, apiVersion: Option[Short] = None): ApiVersionsResponse = { + val response = connectAndSend(request, ApiKeys.API_VERSIONS, apiVersion = apiVersion) + ApiVersionsResponse.parse(response, 0) } } diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index b864e5d0875fb..a26bc2e242413 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -26,8 +26,9 @@ import kafka.integration.KafkaServerTestHarness import kafka.network.SocketServer import kafka.utils._ import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.protocol.types.Struct import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} -import org.apache.kafka.common.requests.{AbstractRequest, RequestHeader, ResponseHeader} +import org.apache.kafka.common.requests.{AbstractRequest, AbstractRequestResponse, RequestHeader, ResponseHeader} abstract class BaseRequestTest extends KafkaServerTestHarness { private var correlationId = 0 @@ -97,41 +98,64 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { } /** - * - * @param request - * @param apiKey * @param destination An optional SocketServer ot send the request to. If not set, any available server is used. * @param protocol An optional SecurityProtocol to use. If not set, PLAINTEXT is used. - * @return + * @return A ByteBuffer containing the response (without the response header) */ - def send(request: AbstractRequest, apiKey: ApiKeys, - destination: SocketServer = anySocketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): ByteBuffer = { + def connectAndSend(request: AbstractRequest, apiKey: ApiKeys, + destination: SocketServer = anySocketServer, + apiVersion: Option[Short] = None, + protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): ByteBuffer = { val socket = connect(destination, protocol) - try { - send(request, apiKey, socket) - } finally { - socket.close() - } + try send(request, apiKey, socket, apiVersion) + finally socket.close() } /** - * Serializes and send the request to the given api. + * @param destination An optional SocketServer ot send the request to. If not set, any available server is used. + * @param protocol An optional SecurityProtocol to use. If not set, PLAINTEXT is used. + * @return A ByteBuffer containing the response (without the response header). + */ + def connectAndSendStruct(requestStruct: Struct, apiKey: ApiKeys, apiVersion: Short, + destination: SocketServer = anySocketServer, + protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): ByteBuffer = { + val socket = connect(destination, protocol) + try sendStruct(requestStruct, apiKey, socket, apiVersion) + finally socket.close() + } + + /** + * Serializes and sends the request to the given api. * A ByteBuffer containing the response is returned. */ - def send(request: AbstractRequest, apiKey: ApiKeys, socket: Socket): ByteBuffer = { - correlationId += 1 - val serializedBytes = { - val header = new RequestHeader(apiKey.id, request.version, "client-id", correlationId) - val byteBuffer = ByteBuffer.allocate(header.sizeOf() + request.sizeOf) - header.writeTo(byteBuffer) - request.writeTo(byteBuffer) - byteBuffer.array() - } + def send(request: AbstractRequest, apiKey: ApiKeys, socket: Socket, apiVersion: Option[Short] = None): ByteBuffer = { + val header = nextRequestHeader(apiKey, apiVersion.getOrElse(request.version)) + val serializedBytes = request.serialize(header).array + val response = requestAndReceive(socket, serializedBytes) + skipResponseHeader(response) + } + /** + * Serializes and sends the requestStruct to the given api. + * A ByteBuffer containing the response (without the response header) is returned. + */ + def sendStruct(requestStruct: Struct, apiKey: ApiKeys, socket: Socket, apiVersion: Short): ByteBuffer = { + val header = nextRequestHeader(apiKey, apiVersion) + val serializedBytes = AbstractRequestResponse.serialize(header.toStruct, requestStruct).array val response = requestAndReceive(socket, serializedBytes) + skipResponseHeader(response) + } + private def skipResponseHeader(response: Array[Byte]): ByteBuffer = { val responseBuffer = ByteBuffer.wrap(response) - ResponseHeader.parse(responseBuffer) // Parse the header to ensure its valid and move the buffer forward + // Parse the header to ensure its valid and move the buffer forward + ResponseHeader.parse(responseBuffer) responseBuffer } + + def nextRequestHeader(apiKey: ApiKeys, apiVersion: Short): RequestHeader = { + correlationId += 1 + new RequestHeader(apiKey.id, apiVersion, "client-id", correlationId) + } + } diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index 6efa189a6c828..4ab952032a4cf 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -116,30 +116,27 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { // Duplicate val singleRequest = new CreateTopicsRequest.Builder(Map("duplicate-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort)).asJava, 1000).build() - val duplicateRequest = duplicateFirstTopic(singleRequest) - assertFalse("Request doesn't have duplicate topics", duplicateRequest.duplicateTopics().isEmpty) - validateErrorCreateTopicsRequests(duplicateRequest, Map("duplicate-topic" -> error(Errors.INVALID_REQUEST, - Some("""Create topics request from client `client-id` contains multiple entries for the following topics: duplicate-topic""")))) + validateErrorCreateTopicsRequests(singleRequest, Map("duplicate-topic" -> error(Errors.INVALID_REQUEST, + Some("""Create topics request from client `client-id` contains multiple entries for the following topics: duplicate-topic"""))), + requestStruct = Some(toStructWithDuplicateFirstTopic(singleRequest))) // Duplicate Partial with validateOnly val doubleRequestValidateOnly = new CreateTopicsRequest.Builder(Map( "duplicate-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort), "other-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort)).asJava, 1000, true).build() - val duplicateDoubleRequestValidateOnly = duplicateFirstTopic(doubleRequestValidateOnly) - assertFalse("Request doesn't have duplicate topics", duplicateDoubleRequestValidateOnly.duplicateTopics.isEmpty) - validateErrorCreateTopicsRequests(duplicateDoubleRequestValidateOnly, Map( + validateErrorCreateTopicsRequests(doubleRequestValidateOnly, Map( "duplicate-topic" -> error(Errors.INVALID_REQUEST), - "other-topic" -> error(Errors.NONE)), checkErrorMessage = false) + "other-topic" -> error(Errors.NONE)), checkErrorMessage = false, + requestStruct = Some(toStructWithDuplicateFirstTopic(doubleRequestValidateOnly))) // Duplicate Partial val doubleRequest = new CreateTopicsRequest.Builder(Map( "duplicate-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort), "other-topic" -> new CreateTopicsRequest.TopicDetails(1, 1.toShort)).asJava, 1000).build() - val duplicateDoubleRequest = duplicateFirstTopic(doubleRequest) - assertFalse("Request doesn't have duplicate topics", duplicateDoubleRequest.duplicateTopics.isEmpty) - validateErrorCreateTopicsRequests(duplicateDoubleRequest, Map( + validateErrorCreateTopicsRequests(doubleRequest, Map( "duplicate-topic" -> error(Errors.INVALID_REQUEST), - "other-topic" -> error(Errors.NONE)), checkErrorMessage = false) + "other-topic" -> error(Errors.NONE)), checkErrorMessage = false, + requestStruct = Some(toStructWithDuplicateFirstTopic(doubleRequest))) // Partitions/ReplicationFactor and ReplicaAssignment val assignments = replicaAssignmentToJava(Map(0 -> List(0))) diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala index 9a092d046b48e..9cd53d8e15888 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala @@ -112,12 +112,12 @@ class DeleteTopicsRequestTest extends BaseRequestTest { } private def sendDeleteTopicsRequest(request: DeleteTopicsRequest, socketServer: SocketServer = controllerSocketServer): DeleteTopicsResponse = { - val response = send(request, ApiKeys.DELETE_TOPICS, socketServer) + val response = connectAndSend(request, ApiKeys.DELETE_TOPICS, socketServer) DeleteTopicsResponse.parse(response, request.version) } private def sendMetadataRequest(request: MetadataRequest): MetadataResponse = { - val response = send(request, ApiKeys.METADATA) - MetadataResponse.parse(response) + val response = connectAndSend(request, ApiKeys.METADATA) + MetadataResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 1f9e18ba805fd..2d4a22a473338 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -114,15 +114,15 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val correlationId = -1 TestUtils.createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = servers) + val version = 2: Short val serializedBytes = { - val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, 2, null, correlationId) + val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, version, null, correlationId) val messageBytes = "message".getBytes val records = MemoryRecords.readableRecords(ByteBuffer.wrap(messageBytes)) - val request = new ProduceRequest.Builder( - 1, 10000, Map(topicPartition -> records).asJava).build() - val byteBuffer = ByteBuffer.allocate(headerBytes.length + request.sizeOf) + val request = new ProduceRequest.Builder(1, 10000, Map(topicPartition -> records).asJava).build() + val byteBuffer = ByteBuffer.allocate(headerBytes.length + request.toStruct.sizeOf) byteBuffer.put(headerBytes) - request.writeTo(byteBuffer) + request.toStruct.writeTo(byteBuffer) byteBuffer.array() } @@ -130,13 +130,13 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val responseBuffer = ByteBuffer.wrap(response) val responseHeader = ResponseHeader.parse(responseBuffer) - val produceResponse = ProduceResponse.parse(responseBuffer) + val produceResponse = ProduceResponse.parse(responseBuffer, version) - assertEquals("The response should parse completely", 0, responseBuffer.remaining()) - assertEquals("The correlationId should match request", correlationId, responseHeader.correlationId()) - assertEquals("One partition response should be returned", 1, produceResponse.responses().size()) + assertEquals("The response should parse completely", 0, responseBuffer.remaining) + assertEquals("The correlationId should match request", correlationId, responseHeader.correlationId) + assertEquals("One partition response should be returned", 1, produceResponse.responses.size) - val partitionResponse = produceResponse.responses().get(topicPartition) + val partitionResponse = produceResponse.responses.get(topicPartition) assertNotNull(partitionResponse) assertEquals("There should be no error", Errors.NONE, partitionResponse.error) } diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 38113609434f8..64be5b3088893 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -24,7 +24,7 @@ import kafka.utils.TestUtils import kafka.utils.TestUtils._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils} import org.apache.kafka.common.record.LogEntry import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer @@ -56,7 +56,7 @@ class FetchRequestTest extends BaseRequestTest { private def createFetchRequest(maxResponseBytes: Int, maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], offsetMap: Map[TopicPartition, Long] = Map.empty): FetchRequest = - new FetchRequest.Builder(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, topicPartitions, offsetMap)) + FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, topicPartitions, offsetMap)) .setMaxBytes(maxResponseBytes).build() private def createPartitionMap(maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], @@ -69,8 +69,8 @@ class FetchRequestTest extends BaseRequestTest { } private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = { - val response = send(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) - FetchResponse.parse(response) + val response = connectAndSend(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) + FetchResponse.parse(response, ProtoUtils.latestVersion(ApiKeys.FETCH.id)) } @Test @@ -156,10 +156,9 @@ class FetchRequestTest extends BaseRequestTest { val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head producer.send(new ProducerRecord(topicPartition.topic, topicPartition.partition, "key", new String(new Array[Byte](maxPartitionBytes + 1)))).get - val fetchRequestBuilder = new FetchRequest.Builder( - Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, Seq(topicPartition))). - setVersion(2) - val fetchResponse = sendFetchRequest(leaderId, fetchRequestBuilder.build()) + val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, + createPartitionMap(maxPartitionBytes, Seq(topicPartition))).build(2) + val fetchResponse = sendFetchRequest(leaderId, fetchRequest) val partitionData = fetchResponse.responseData.get(topicPartition) assertEquals(Errors.NONE, partitionData.error) assertTrue(partitionData.highWatermark > 0) diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 99a95adb9e455..1b5007dbd41f7 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -22,7 +22,7 @@ import util.Arrays.asList import kafka.common.BrokerEndPointNotAvailableException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} import org.apache.kafka.common.requests.{PartitionState, UpdateMetadataRequest} import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint} import org.junit.Test @@ -69,8 +69,9 @@ class MetadataCacheTest { new TopicPartition(topic, 1) -> new PartitionState(controllerEpoch, 1, 1, asList(1), zkVersion, asSet(1)), new TopicPartition(topic, 2) -> new PartitionState(controllerEpoch, 2, 2, asList(2), zkVersion, asSet(2))) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) for (securityProtocol <- Seq(SecurityProtocol.PLAINTEXT, SecurityProtocol.SSL)) { @@ -120,8 +121,9 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asSet(0))) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName) @@ -163,8 +165,9 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) // Validate errorUnavailableEndpoints = false @@ -222,8 +225,9 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, + partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) // Validate errorUnavailableEndpoints = false @@ -273,8 +277,9 @@ class MetadataCacheTest { val isr = asList[Integer](0, 1) val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, + brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) try { @@ -305,8 +310,9 @@ class MetadataCacheTest { val isr = asList[Integer](0, 1) val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) - val updateMetadataRequest = new UpdateMetadataRequest.Builder( - 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() + val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, + brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) } diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index f3bb912e0260b..ed0e8051ffd0b 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -36,14 +36,14 @@ class MetadataRequestTest extends BaseRequestTest { @Test def testClusterIdWithRequestVersion1() { - val v1MetadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val v1MetadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) val v1ClusterId = v1MetadataResponse.clusterId assertNull(s"v1 clusterId should be null", v1ClusterId) } @Test def testClusterIdIsValid() { - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(2.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(2.toShort)) isValidClusterId(metadataResponse.clusterId) } @@ -51,7 +51,7 @@ class MetadataRequestTest extends BaseRequestTest { def testControllerId() { val controllerServer = servers.find(_.kafkaController.isActive).get val controllerId = controllerServer.config.brokerId - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) assertEquals("Controller id should match the active controller", controllerId, metadataResponse.controller.id) @@ -64,14 +64,14 @@ class MetadataRequestTest extends BaseRequestTest { val controllerId2 = controllerServer2.config.brokerId assertNotEquals("Controller id should switch to a new broker", controllerId, controllerId2) TestUtils.waitUntilTrue(() => { - val metadataResponse2 = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse2 = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) metadataResponse2.controller != null && controllerServer2.apis.brokerId == metadataResponse2.controller.id }, "Controller id should match the active controller after failover", 5000) } @Test def testRack() { - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) // Validate rack matches what's set in generateConfigs() above metadataResponse.brokers.asScala.foreach { broker => assertEquals("Rack information should match config", s"rack/${broker.id}", broker.rack) @@ -86,7 +86,7 @@ class MetadataRequestTest extends BaseRequestTest { TestUtils.createTopic(zkUtils, internalTopic, 3, 2, servers) TestUtils.createTopic(zkUtils, notInternalTopic, 3, 2, servers) - val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) assertTrue("Response should have no errors", metadataResponse.errors.isEmpty) val topicMetadata = metadataResponse.topicMetadata.asScala @@ -124,7 +124,7 @@ class MetadataRequestTest extends BaseRequestTest { assertEquals("V0 Response should have 2 (all) topics", 2, metadataResponseV0.topicMetadata.size()) // v1, Null represents all topics - val metadataResponseV1 = sendMetadataRequest(MetadataRequest.allTopics(1.toShort)) + val metadataResponseV1 = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) assertTrue("V1 Response should have no errors", metadataResponseV1.errors.isEmpty) assertEquals("V1 Response should have 2 (all) topics", 2, metadataResponseV1.topicMetadata.size()) } @@ -177,7 +177,7 @@ class MetadataRequestTest extends BaseRequestTest { } private def sendMetadataRequest(request: MetadataRequest): MetadataResponse = { - val response = send(request, ApiKeys.METADATA) + val response = connectAndSend(request, ApiKeys.METADATA) MetadataResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index b05be9d580531..81118fa84b04e 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -88,8 +88,8 @@ class ProduceRequestTest extends BaseRequestTest { } private def sendProduceRequest(leaderId: Int, request: ProduceRequest): ProduceResponse = { - val response = send(request, ApiKeys.PRODUCE, destination = brokerSocketServer(leaderId)) - ProduceResponse.parse(response) + val response = connectAndSend(request, ApiKeys.PRODUCE, destination = brokerSocketServer(leaderId)) + ProduceResponse.parse(response, request.version) } } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 927ace9b42b45..92a518dc852c5 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -20,14 +20,13 @@ import java.io.IOException import java.net.Socket import java.util.Collections -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} import org.apache.kafka.common.requests.SaslHandshakeRequest import org.apache.kafka.common.requests.SaslHandshakeResponse import org.junit.Test import org.junit.Assert._ import kafka.api.SaslTestHarness -import org.apache.kafka.common.protocol.types.Struct class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT @@ -42,8 +41,7 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { def testApiVersionsRequestBeforeSaslHandshakeRequest() { val plaintextSocket = connect(protocol = securityProtocol) try { - val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, - new ApiVersionsRequest.Builder().setVersion(0).build()) + val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest.Builder().build(0)) ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse) sendSaslHandshakeRequestValidateResponse(plaintextSocket) } finally { @@ -57,8 +55,7 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { try { sendSaslHandshakeRequestValidateResponse(plaintextSocket) try { - sendApiVersionsRequest(plaintextSocket, - new ApiVersionsRequest.Builder().setVersion(0).build()) + sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest.Builder().build(0)) fail("Versions Request during Sasl handshake did not fail") } catch { case _: IOException => // expected exception @@ -72,12 +69,10 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { def testApiVersionsRequestWithUnsupportedVersion() { val plaintextSocket = connect(protocol = securityProtocol) try { - val apiVersionsRequest = new ApiVersionsRequest( - new Struct(ProtoUtils.requestSchema(ApiKeys.API_VERSIONS.id, 0)), Short.MaxValue); - val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, apiVersionsRequest) + val apiVersionsRequest = new ApiVersionsRequest(0) + val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, apiVersionsRequest, Some(Short.MaxValue)) assertEquals(Errors.UNSUPPORTED_VERSION, apiVersionsResponse.error) - val apiVersionsResponse2 = sendApiVersionsRequest(plaintextSocket, - new ApiVersionsRequest.Builder().setVersion(0).build()) + val apiVersionsResponse2 = sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest.Builder().build(0)) ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse2) sendSaslHandshakeRequestValidateResponse(plaintextSocket) } finally { @@ -85,15 +80,17 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { } } - private def sendApiVersionsRequest(socket: Socket, request: ApiVersionsRequest): ApiVersionsResponse = { - val response = send(request, ApiKeys.API_VERSIONS, socket) - ApiVersionsResponse.parse(response) + private def sendApiVersionsRequest(socket: Socket, request: ApiVersionsRequest, + apiVersion: Option[Short] = None): ApiVersionsResponse = { + val response = send(request, ApiKeys.API_VERSIONS, socket, apiVersion) + ApiVersionsResponse.parse(response, request.version) } private def sendSaslHandshakeRequestValidateResponse(socket: Socket) { - val response = send(new SaslHandshakeRequest("PLAIN"), ApiKeys.SASL_HANDSHAKE, socket) - val handshakeResponse = SaslHandshakeResponse.parse(response) + val request = new SaslHandshakeRequest("PLAIN") + val response = send(request, ApiKeys.SASL_HANDSHAKE, socket) + val handshakeResponse = SaslHandshakeResponse.parse(response, request.version) assertEquals(Errors.NONE, handshakeResponse.error) - assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms()) + assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms) } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index 69b3c46d162bf..539965390e431 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -106,8 +106,8 @@ public MetadataResponse fetchMetadata() { MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, new ArrayList(), new ArrayList()); MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(Errors.NONE, topic, true, Collections.singletonList(partitionMetadata)); MetadataResponse response = new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, - Collections.singletonList(topicMetadata), 0); + Collections.singletonList(topicMetadata)); return response; } } -} \ No newline at end of file +} From 1f1e794ad0025bb2a33b7d8378a481f224b3bccc Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Fri, 17 Feb 2017 14:41:28 -0800 Subject: [PATCH 009/101] KAFKA-4317: Regularly checkpoint StateStore changelog offsets Currently the checkpoint file is deleted at state store initialization and it is only ever written again during a clean shutdown. This can result in significant delays during restarts as the entire store needs to be loaded from the changelog. We can mitigate against this by frequently checkpointing the offsets. The checkpointing happens only during the commit phase, i.e, after we have manually flushed the store and the producer. So we guarantee that the checkpointed offsets are never greater than what has been flushed. In the event of hard failure we can recover by reading the checkpoints and consuming from the stored offsets. Author: Damian Guy Reviewers: Eno Thereska, Matthias J. Sax, Guozhang Wang Closes #2471 from dguy/kafka-4317 --- .../processor/internals/AbstractTask.java | 18 +- .../processor/internals/Checkpointable.java | 27 +++ .../internals/GlobalStateManagerImpl.java | 16 +- .../internals/GlobalStateUpdateTask.java | 3 +- .../internals/ProcessorStateManager.java | 57 +++--- .../processor/internals/StandbyTask.java | 23 +-- .../processor/internals/StateManager.java | 4 +- .../processor/internals/StreamTask.java | 5 +- .../internals/InMemoryKeyValueStore.java | 2 +- .../processor/internals/AbstractTaskTest.java | 3 +- .../internals/GlobalStateManagerImplTest.java | 49 ++++- .../internals/GlobalStateTaskTest.java | 16 +- .../internals/ProcessorStateManagerTest.java | 186 ++++++++++++------ .../processor/internals/StandbyTaskTest.java | 51 ++++- .../processor/internals/StateManagerStub.java | 7 +- .../processor/internals/StreamTaskTest.java | 60 ++++++ .../kafka/test/GlobalStateManagerStub.java | 7 +- .../test/ProcessorTopologyTestDriver.java | 3 +- 18 files changed, 401 insertions(+), 136 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index 2a040ba508f76..8de5d2304d05a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -52,14 +52,14 @@ public abstract class AbstractTask { /** * @throws ProcessorStateException if the state manager cannot be created */ - protected AbstractTask(TaskId id, - String applicationId, - Collection partitions, - ProcessorTopology topology, - Consumer consumer, - Consumer restoreConsumer, - boolean isStandby, - StateDirectory stateDirectory, + protected AbstractTask(final TaskId id, + final String applicationId, + final Collection partitions, + final ProcessorTopology topology, + final Consumer consumer, + final Consumer restoreConsumer, + final boolean isStandby, + final StateDirectory stateDirectory, final ThreadCache cache) { this.id = id; this.applicationId = applicationId; @@ -70,7 +70,7 @@ protected AbstractTask(TaskId id, // create the processor state manager try { - this.stateMgr = new ProcessorStateManager(id, partitions, restoreConsumer, isStandby, stateDirectory, topology.storeToChangelogTopic()); + stateMgr = new ProcessorStateManager(id, partitions, restoreConsumer, isStandby, stateDirectory, topology.storeToChangelogTopic()); } catch (IOException e) { throw new ProcessorStateException(String.format("task [%s] Error while creating the state manager", id), e); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java new file mode 100644 index 0000000000000..7b02d5b934365 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +// Interface to indicate that an object has associated partition offsets that can be checkpointed +interface Checkpointable { + void checkpoint(final Map offsets); + Map checkpointed(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 75349931aa199..3819bb5a09d70 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -58,11 +58,11 @@ public class GlobalStateManagerImpl implements GlobalStateManager { private final File baseDir; private final OffsetCheckpoint checkpoint; private final Set globalStoreNames = new HashSet<>(); - private HashMap checkpointableOffsets; + private final Map checkpointableOffsets = new HashMap<>(); public GlobalStateManagerImpl(final ProcessorTopology topology, - final Consumer consumer, - final StateDirectory stateDirectory) { + final Consumer consumer, + final StateDirectory stateDirectory) { this.topology = topology; this.consumer = consumer; this.stateDirectory = stateDirectory; @@ -81,8 +81,7 @@ public Set initialize(final InternalProcessorContext processorContext) { } try { - this.checkpointableOffsets = new HashMap<>(checkpoint.read()); - checkpoint.delete(); + this.checkpointableOffsets.putAll(checkpoint.read()); } catch (IOException e) { try { stateDirectory.unlockGlobalState(); @@ -220,13 +219,14 @@ public void close(final Map offsets) throws IOException { if (closeFailed.length() > 0) { throw new ProcessorStateException("Exceptions caught during close of 1 or more global state stores\n" + closeFailed); } - writeCheckpoints(offsets); + checkpoint(offsets); } finally { stateDirectory.unlockGlobalState(); } } - private void writeCheckpoints(final Map offsets) { + @Override + public void checkpoint(final Map offsets) { if (!offsets.isEmpty()) { checkpointableOffsets.putAll(offsets); try { @@ -238,7 +238,7 @@ private void writeCheckpoints(final Map offsets) { } @Override - public Map checkpointedOffsets() { + public Map checkpointed() { return Collections.unmodifiableMap(checkpointableOffsets); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index 40f2a3c826aa8..6da37e4c1c003 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -67,7 +67,7 @@ public Map initialize() { } initTopology(); processorContext.initialized(); - return stateMgr.checkpointedOffsets(); + return stateMgr.checkpointed(); } @@ -89,6 +89,7 @@ public void update(final ConsumerRecord record) { public void flushState() { stateMgr.flush(processorContext); + stateMgr.checkpoint(offsets); } public void close() throws IOException { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 1c786e30a88ea..0e8caa2b76ed8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -68,6 +68,7 @@ public class ProcessorStateManager implements StateManager { // TODO: this map does not work with customized grouper where multiple partitions // of the same topic can be assigned to the same topic. private final Map partitionForTopic; + private final OffsetCheckpoint checkpoint; /** * @throws LockException if the state directory cannot be locked because another thread holds the lock @@ -103,11 +104,8 @@ public ProcessorStateManager(final TaskId taskId, } // load the checkpoint information - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); this.checkpointedOffsets = new HashMap<>(checkpoint.read()); - - // delete the checkpoint file after finish loading its stored offsets - checkpoint.delete(); } @@ -250,7 +248,7 @@ private void restoreActiveState(String topicName, StateRestoreCallback stateRest } } - public Map checkpointedOffsets() { + public Map checkpointed() { Map partitionsAndOffsets = new HashMap<>(); for (Map.Entry entry : restoreCallbacks.entrySet()) { @@ -347,29 +345,7 @@ public void close(Map ackedOffsets) throws IOException { } if (ackedOffsets != null) { - Map checkpointOffsets = new HashMap<>(); - for (String storeName : stores.keySet()) { - // only checkpoint the offset to the offsets file if - // it is persistent AND changelog enabled - if (stores.get(storeName).persistent() && storeToChangelogTopic.containsKey(storeName)) { - String changelogTopic = storeToChangelogTopic.get(storeName); - TopicPartition topicPartition = new TopicPartition(changelogTopic, getPartition(storeName)); - Long offset = ackedOffsets.get(topicPartition); - - if (offset != null) { - // store the last offset + 1 (the log position after restoration) - checkpointOffsets.put(topicPartition, offset + 1); - } else { - // if no record was produced. we need to check the restored offset. - offset = restoredOffsets.get(topicPartition); - if (offset != null) - checkpointOffsets.put(topicPartition, offset); - } - } - } - // write the checkpoint file before closing, to indicate clean shutdown - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); - checkpoint.write(checkpointOffsets); + checkpoint(ackedOffsets); } } @@ -379,6 +355,31 @@ public void close(Map ackedOffsets) throws IOException { } } + // write the checkpoint + @Override + public void checkpoint(final Map ackedOffsets) { + for (String storeName : stores.keySet()) { + // only checkpoint the offset to the offsets file if + // it is persistent AND changelog enabled + if (stores.get(storeName).persistent() && storeToChangelogTopic.containsKey(storeName)) { + final String changelogTopic = storeToChangelogTopic.get(storeName); + final TopicPartition topicPartition = new TopicPartition(changelogTopic, getPartition(storeName)); + if (ackedOffsets.containsKey(topicPartition)) { + // store the last offset + 1 (the log position after restoration) + checkpointedOffsets.put(topicPartition, ackedOffsets.get(topicPartition) + 1); + } else if (restoredOffsets.containsKey(topicPartition)) { + checkpointedOffsets.put(topicPartition, restoredOffsets.get(topicPartition)); + } + } + } + // write the checkpoint file before closing, to indicate clean shutdown + try { + checkpoint.write(checkpointedOffsets); + } catch (IOException e) { + log.warn("Failed to write checkpoint file to {}", new File(baseDir, CHECKPOINT_FILE_NAME), e); + } + } + private int getPartition(String topic) { TopicPartition partition = partitionForTopic.get(topic); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 4437a1955e700..a27098c34f22e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -51,14 +51,15 @@ public class StandbyTask extends AbstractTask { * @param metrics the {@link StreamsMetrics} created by the thread * @param stateDirectory the {@link StateDirectory} created by the thread */ - public StandbyTask(TaskId id, - String applicationId, - Collection partitions, - ProcessorTopology topology, - Consumer consumer, - Consumer restoreConsumer, - StreamsConfig config, - StreamsMetrics metrics, final StateDirectory stateDirectory) { + public StandbyTask(final TaskId id, + final String applicationId, + final Collection partitions, + final ProcessorTopology topology, + final Consumer consumer, + final Consumer restoreConsumer, + final StreamsConfig config, + final StreamsMetrics metrics, + final StateDirectory stateDirectory) { super(id, applicationId, partitions, topology, consumer, restoreConsumer, true, stateDirectory, null); // initialize the topology with its own context @@ -67,9 +68,9 @@ public StandbyTask(TaskId id, log.info("standby-task [{}] Initializing state stores", id()); initializeStateStores(); - ((StandbyContextImpl) this.processorContext).initialized(); + this.processorContext.initialized(); - this.checkpointedOffsets = Collections.unmodifiableMap(stateMgr.checkpointedOffsets()); + this.checkpointedOffsets = Collections.unmodifiableMap(stateMgr.checkpointed()); } public Map checkpointedOffsets() { @@ -92,7 +93,7 @@ public List> update(TopicPartition partition, Lis public void commit() { log.debug("standby-task [{}] Committing its state", id()); stateMgr.flush(processorContext); - + stateMgr.checkpoint(Collections.emptyMap()); // reinitialize offset limits initializeOffsetLimits(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java index 7343c857fa68b..3102b77163086 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java @@ -24,7 +24,7 @@ import java.io.IOException; import java.util.Map; -interface StateManager { +interface StateManager extends Checkpointable { File baseDir(); void register(final StateStore store, final boolean loggingEnabled, final StateRestoreCallback stateRestoreCallback); @@ -36,6 +36,4 @@ interface StateManager { StateStore getGlobalStore(final String name); StateStore getStore(final String name); - - Map checkpointedOffsets(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index be77856450c07..d95ac4bac6fc1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -76,8 +76,9 @@ public void run() { log.trace("{} Start flushing its producer's sent records upon committing its state", logPrefix); // 2) flush produced records in the downstream and change logs of local states recordCollector.flush(); - - // 3) commit consumed offsets if it is dirty already + // 3) write checkpoints for any local state + stateMgr.checkpoint(recordCollectorOffsets()); + // 4) commit consumed offsets if it is dirty already commitOffsets(); } }; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index fe501523af1aa..28dea79acf226 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -39,7 +39,7 @@ public class InMemoryKeyValueStore implements KeyValueStore { private StateSerdes serdes; - InMemoryKeyValueStore(final String name, final Serde keySerde, final Serde valueSerde) { + public InMemoryKeyValueStore(final String name, final Serde keySerde, final Serde valueSerde) { this.name = name; this.keySerde = keySerde; this.valueSerde = valueSerde; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java index fc0953bb13e39..f288f984d1ab1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java @@ -60,6 +60,7 @@ public void shouldThrowWakeupExceptionOnInitializeOffsetsWhenWakeupException() t } private AbstractTask createTask(final Consumer consumer) { + final MockTime time = new MockTime(); return new AbstractTask(new TaskId(0, 0), "app", Collections.singletonList(new TopicPartition("t", 0)), @@ -72,7 +73,7 @@ private AbstractTask createTask(final Consumer consumer) { consumer, consumer, false, - new StateDirectory("app", TestUtils.tempDirectory().getPath(), new MockTime()), + new StateDirectory("app", TestUtils.tempDirectory().getPath(), time), new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))) { @Override public void commit() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java index 168b3004eeb86..062079fefcbea 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java @@ -50,6 +50,8 @@ import java.util.Map; import java.util.Set; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -125,15 +127,15 @@ public void shouldReadCheckpointOffsets() throws Exception { final Map expected = writeCheckpoint(); stateManager.initialize(context); - final Map offsets = stateManager.checkpointedOffsets(); + final Map offsets = stateManager.checkpointed(); assertEquals(expected, offsets); } @Test - public void shouldDeleteCheckpointFileAfteLoaded() throws Exception { + public void shouldNotDeleteCheckpointFileAfterLoaded() throws Exception { writeCheckpoint(); stateManager.initialize(context); - assertFalse(checkpointFile.exists()); + assertTrue(checkpointFile.exists()); } @Test(expected = StreamsException.class) @@ -168,7 +170,7 @@ public void shouldThrowIllegalArgumentIfTryingToRegisterStoreThatIsNotGlobal() t } @Test - public void shouldThrowIllegalArgumenExceptionIfAttemptingToRegisterStoreTwice() throws Exception { + public void shouldThrowIllegalArgumentExceptionIfAttemptingToRegisterStoreTwice() throws Exception { stateManager.initialize(context); initializeConsumer(2, 1, t1); stateManager.register(store1, false, new TheStateRestoreCallback()); @@ -271,9 +273,7 @@ public void shouldWriteCheckpointsOnClose() throws Exception { stateManager.register(store1, false, stateRestoreCallback); final Map expected = Collections.singletonMap(t1, 25L); stateManager.close(expected); - final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint(new File(stateManager.baseDir(), - ProcessorStateManager.CHECKPOINT_FILE_NAME)); - final Map result = offsetCheckpoint.read(); + final Map result = readOffsetsCheckpoint(); assertEquals(expected, result); } @@ -376,6 +376,41 @@ public void shouldReleaseLockIfExceptionWhenLoadingCheckpoints() throws Exceptio } } + @Test + public void shouldCheckpointOffsets() throws Exception { + final Map offsets = Collections.singletonMap(t1, 25L); + stateManager.initialize(context); + + stateManager.checkpoint(offsets); + + final Map result = readOffsetsCheckpoint(); + assertThat(result, equalTo(offsets)); + assertThat(stateManager.checkpointed(), equalTo(offsets)); + } + + @Test + public void shouldNotRemoveOffsetsOfUnUpdatedTablesDuringCheckpoint() throws Exception { + stateManager.initialize(context); + final TheStateRestoreCallback stateRestoreCallback = new TheStateRestoreCallback(); + initializeConsumer(10, 1, t1); + stateManager.register(store1, false, stateRestoreCallback); + initializeConsumer(20, 1, t2); + stateManager.register(store2, false, stateRestoreCallback); + + final Map initialCheckpoint = stateManager.checkpointed(); + stateManager.checkpoint(Collections.singletonMap(t1, 101L)); + + final Map updatedCheckpoint = stateManager.checkpointed(); + assertThat(updatedCheckpoint.get(t2), equalTo(initialCheckpoint.get(t2))); + assertThat(updatedCheckpoint.get(t1), equalTo(101L)); + } + + private Map readOffsetsCheckpoint() throws IOException { + final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint(new File(stateManager.baseDir(), + ProcessorStateManager.CHECKPOINT_FILE_NAME)); + return offsetCheckpoint.read(); + } + @Test public void shouldThrowLockExceptionIfIOExceptionCaughtWhenTryingToLockStateDir() throws Exception { stateManager = new GlobalStateManagerImpl(topology, consumer, new StateDirectory("appId", stateDirPath, time) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java index df0b73ca8a4cd..66999bc8efbe1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java @@ -38,6 +38,8 @@ import java.util.Map; import java.util.Set; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -137,7 +139,19 @@ public void shouldCloseStateManagerWithOffsets() throws Exception { globalStateTask.initialize(); globalStateTask.update(new ConsumerRecord<>("t1", 1, 51, "foo".getBytes(), "foo".getBytes())); globalStateTask.close(); - assertEquals(expectedOffsets, stateMgr.checkpointedOffsets()); + assertEquals(expectedOffsets, stateMgr.checkpointed()); assertTrue(stateMgr.closed); } + + @Test + public void shouldCheckpointOffsetsWhenStateIsFlushed() throws Exception { + final Map expectedOffsets = new HashMap<>(); + expectedOffsets.put(t1, 102L); + expectedOffsets.put(t2, 100L); + globalStateTask.initialize(); + globalStateTask.update(new ConsumerRecord<>("t1", 1, 101, "foo".getBytes(), "foo".getBytes())); + globalStateTask.flushState(); + assertThat(stateMgr.checkpointed(), equalTo(expectedOffsets)); + } + } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index b8d51baff8de2..f1d3090b2c026 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -58,6 +59,8 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -71,7 +74,6 @@ public static class MockRestoreConsumer extends MockConsumer { private final Serializer serializer = new IntegerSerializer(); private TopicPartition assignedPartition = null; - private TopicPartition seekPartition = null; private long seekOffset = -1L; private boolean seekToBeginingCalled = false; private boolean seekToEndCalled = false; @@ -162,7 +164,6 @@ public synchronized void seek(TopicPartition partition, long offset) { if (seekOffset >= 0) throw new IllegalStateException("RestoreConsumer: offset already seeked"); - seekPartition = partition; seekOffset = offset; currentOffset = offset; super.seek(partition, offset); @@ -203,6 +204,9 @@ public synchronized void seekToEnd(Collection partitions) { private final String nonPersistentStoreName = "nonPersistentStore"; private final String persistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(applicationId, persistentStoreName); private final String nonPersistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(applicationId, nonPersistentStoreName); + private final MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); + private final MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); + private final TopicPartition persistentStorePartition = new TopicPartition(persistentStoreTopicName, 1); private final String storeName = "mockStateStore"; private final String changelogTopic = ProcessorStateManager.storeChangelogTopic(applicationId, storeName); private final TopicPartition changelogTopicPartition = new TopicPartition(changelogTopic, 0); @@ -210,6 +214,8 @@ public synchronized void seekToEnd(Collection partitions) { private final MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); private final MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(storeName, true); private File baseDir; + private File checkpointFile; + private OffsetCheckpoint checkpoint; private StateDirectory stateDirectory; @@ -217,6 +223,14 @@ public synchronized void seekToEnd(Collection partitions) { public void setup() { baseDir = TestUtils.tempDirectory(); stateDirectory = new StateDirectory(applicationId, baseDir.getPath(), new MockTime()); + checkpointFile = new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME); + checkpoint = new OffsetCheckpoint(checkpointFile); + restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( + new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( + new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) + )); } @After @@ -300,8 +314,6 @@ public void testRegisterPersistentStore() throws IOException { public void testRegisterNonPersistentStore() throws IOException { long lastCheckpointedOffset = 10L; - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); checkpoint.write(Collections.singletonMap(new TopicPartition(persistentStoreTopicName, 2), lastCheckpointedOffset)); @@ -313,8 +325,6 @@ public void testRegisterNonPersistentStore() throws IOException { TopicPartition partition = new TopicPartition(persistentStoreTopicName, 2); restoreConsumer.updateEndOffsets(Collections.singletonMap(partition, 13L)); - MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); // non persistent store - ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 2), noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); @@ -325,7 +335,7 @@ public void testRegisterNonPersistentStore() throws IOException { restoreConsumer.reset(); ArrayList expectedKeys = new ArrayList<>(); - long offset = -1L; + long offset; for (int i = 1; i <= 3; i++) { offset = (long) (i + 100); int key = i; @@ -346,12 +356,13 @@ public void testRegisterNonPersistentStore() throws IOException { } finally { stateMgr.close(Collections.emptyMap()); } - } @Test public void testChangeLogOffsets() throws IOException { final TaskId taskId = new TaskId(0, 0); + final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint( + new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME)); long lastCheckpointedOffset = 10L; String storeName1 = "store1"; String storeName2 = "store2"; @@ -366,10 +377,7 @@ public void testChangeLogOffsets() throws IOException { storeToChangelogTopic.put(storeName2, storeTopicName2); storeToChangelogTopic.put(storeName3, storeTopicName3); - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME)); - checkpoint.write(Collections.singletonMap(new TopicPartition(storeTopicName1, 0), lastCheckpointedOffset)); - - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + offsetCheckpoint.write(Collections.singletonMap(new TopicPartition(storeTopicName1, 0), lastCheckpointedOffset)); restoreConsumer.updatePartitions(storeTopicName1, Utils.mkList( new PartitionInfo(storeTopicName1, 0, Node.noNode(), new Node[0], new Node[0]) @@ -406,7 +414,7 @@ public void testChangeLogOffsets() throws IOException { stateMgr.register(store2, true, store2.stateRestoreCallback); stateMgr.register(store3, true, store3.stateRestoreCallback); - Map changeLogOffsets = stateMgr.checkpointedOffsets(); + Map changeLogOffsets = stateMgr.checkpointed(); assertEquals(3, changeLogOffsets.size()); assertTrue(changeLogOffsets.containsKey(partition1)); @@ -424,20 +432,12 @@ public void testChangeLogOffsets() throws IOException { @Test public void testGetStore() throws IOException { - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); - - restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( - new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); - - MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - - ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + final ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); try { - stateMgr.register(mockStateStore, true, mockStateStore.stateRestoreCallback); + stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); assertNull(stateMgr.getStore("noSuchStore")); - assertEquals(mockStateStore, stateMgr.getStore(nonPersistentStoreName)); + assertEquals(nonPersistentStore, stateMgr.getStore(nonPersistentStoreName)); } finally { stateMgr.close(Collections.emptyMap()); @@ -446,30 +446,15 @@ public void testGetStore() throws IOException { @Test public void testFlushAndClose() throws IOException { - final TaskId taskId = new TaskId(0, 1); - File checkpointFile = new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME); // write an empty checkpoint file - OffsetCheckpoint oldCheckpoint = new OffsetCheckpoint(checkpointFile); - oldCheckpoint.write(Collections.emptyMap()); - - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); - - restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( - new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); - restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( - new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); + checkpoint.write(Collections.emptyMap()); // set up ack'ed offsets - HashMap ackedOffsets = new HashMap<>(); + final HashMap ackedOffsets = new HashMap<>(); ackedOffsets.put(new TopicPartition(persistentStoreTopicName, 1), 123L); ackedOffsets.put(new TopicPartition(nonPersistentStoreTopicName, 1), 456L); ackedOffsets.put(new TopicPartition(ProcessorStateManager.storeChangelogTopic(applicationId, "otherTopic"), 1), 789L); - MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); - MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); @@ -477,8 +462,8 @@ public void testFlushAndClose() throws IOException { } }); try { - // make sure the checkpoint file is deleted - assertFalse(checkpointFile.exists()); + // make sure the checkpoint file isn't deleted + assertTrue(checkpointFile.exists()); restoreConsumer.reset(); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); @@ -499,39 +484,122 @@ public void testFlushAndClose() throws IOException { assertTrue(checkpointFile.exists()); // the checkpoint file should contain an offset from the persistent store only. - OffsetCheckpoint newCheckpoint = new OffsetCheckpoint(checkpointFile); - Map checkpointedOffsets = newCheckpoint.read(); + final Map checkpointedOffsets = checkpoint.read(); assertEquals(1, checkpointedOffsets.size()); assertEquals(new Long(123L + 1L), checkpointedOffsets.get(new TopicPartition(persistentStoreTopicName, 1))); } @Test public void shouldRegisterStoreWithoutLoggingEnabledAndNotBackedByATopic() throws Exception { - MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, Collections.emptyMap()); - stateMgr.register(mockStateStore, false, mockStateStore.stateRestoreCallback); + stateMgr.register(nonPersistentStore, false, nonPersistentStore.stateRestoreCallback); assertNotNull(stateMgr.getStore(nonPersistentStoreName)); } @Test - public void shouldNotWriteCheckpointsIfAckeOffsetsIsNull() throws Exception { - final File checkpointFile = new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME); - // write an empty checkpoint file - final OffsetCheckpoint oldCheckpoint = new OffsetCheckpoint(checkpointFile); - oldCheckpoint.write(Collections.emptyMap()); + public void shouldNotChangeOffsetsIfAckedOffsetsIsNull() throws Exception { + final Map offsets = Collections.singletonMap(persistentStorePartition, 99L); + checkpoint.write(offsets); - restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( - new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + false, + stateDirectory, + Collections.emptyMap()); + + restoreConsumer.reset(); + stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); + stateMgr.close(null); + final Map read = checkpoint.read(); + assertThat(read, equalTo(offsets)); + } + + @Test + public void shouldWriteCheckpointForPersistentLogEnabledStore() throws Exception { + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + false, + stateDirectory, + Collections.singletonMap(persistentStore.name(), + persistentStoreTopicName)); + restoreConsumer.reset(); + stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); - final MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); - final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + stateMgr.checkpoint(Collections.singletonMap(persistentStorePartition, 10L)); + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.singletonMap(persistentStorePartition, 11L))); + } + + @Test + public void shouldWriteCheckpointForStandbyReplica() throws Exception { + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + true, + stateDirectory, + Collections.singletonMap(persistentStore.name(), + persistentStoreTopicName)); restoreConsumer.reset(); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); - stateMgr.close(null); - assertFalse(checkpointFile.exists()); + final byte[] bytes = Serdes.Integer().serializer().serialize("", 10); + stateMgr.updateStandbyStates(persistentStorePartition, + Collections.singletonList( + new ConsumerRecord<>(persistentStorePartition.topic(), + persistentStorePartition.partition(), + 888L, + bytes, + bytes))); + + stateMgr.checkpoint(Collections.emptyMap()); + + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.singletonMap(persistentStorePartition, 889L))); + + } + + @Test + public void shouldNotWriteCheckpointForNonPersistent() throws Exception { + final TopicPartition topicPartition = new TopicPartition(nonPersistentStoreTopicName, 1); + + restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( + new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) + )); + + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + true, + stateDirectory, + Collections.singletonMap(nonPersistentStoreName, + nonPersistentStoreTopicName)); + + restoreConsumer.reset(); + stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); + stateMgr.checkpoint(Collections.singletonMap(topicPartition, 876L)); + + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.emptyMap())); + } + + @Test + public void shouldNotWriteCheckpointForStoresWithoutChangelogTopic() throws Exception { + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + true, + stateDirectory, + Collections.emptyMap()); + + stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); + + stateMgr.checkpoint(Collections.singletonMap(persistentStorePartition, 987L)); + + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.emptyMap())); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 629e5219a263d..ef4ebcc7c6a8d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -54,6 +55,8 @@ import java.util.Set; import static java.util.Collections.singleton; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -316,7 +319,7 @@ public void shouldNotThrowUnsupportedOperationExceptionWhenInitializingStateStor final String changelogName = "test-application-my-store-changelog"; final List partitions = Utils.mkList(new TopicPartition(changelogName, 0)); consumer.assign(partitions); - Map committedOffsets = new HashMap<>(); + final Map committedOffsets = new HashMap<>(); committedOffsets.put(new TopicPartition(changelogName, 0), new OffsetAndMetadata(0L)); consumer.commitSync(committedOffsets); @@ -327,9 +330,53 @@ public void shouldNotThrowUnsupportedOperationExceptionWhenInitializingStateStor final ProcessorTopology topology = builder.setApplicationId(applicationId).build(0); StreamsConfig config = createConfig(baseDir); new StandbyTask(taskId, applicationId, partitions, topology, consumer, restoreStateConsumer, config, - new MockStreamsMetrics(new Metrics()), stateDirectory); + new MockStreamsMetrics(new Metrics()), stateDirectory); } + + @Test + public void shouldCheckpointStoreOffsetsOnCommit() throws Exception { + consumer.assign(Utils.mkList(ktable)); + final Map committedOffsets = new HashMap<>(); + committedOffsets.put(new TopicPartition(ktable.topic(), ktable.partition()), new OffsetAndMetadata(100L)); + consumer.commitSync(committedOffsets); + + restoreStateConsumer.updatePartitions("ktable1", Utils.mkList( + new PartitionInfo("ktable1", 0, Node.noNode(), new Node[0], new Node[0]))); + + final TaskId taskId = new TaskId(0, 0); + final MockTime time = new MockTime(); + final StreamsConfig config = createConfig(baseDir); + final StandbyTask task = new StandbyTask(taskId, + applicationId, + ktablePartitions, + ktableTopology, + consumer, + restoreStateConsumer, + config, + null, + stateDirectory + ); + + + restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); + + final byte[] serializedValue = Serdes.Integer().serializer().serialize("", 1); + task.update(ktable, Collections.singletonList(new ConsumerRecord<>(ktable.topic(), + ktable.partition(), + 50L, + serializedValue, + serializedValue))); + + time.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)); + task.commit(); + + final Map checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), + ProcessorStateManager.CHECKPOINT_FILE_NAME)).read(); + assertThat(checkpoint, equalTo(Collections.singletonMap(ktable, 51L))); + + } + private List> records(ConsumerRecord... recs) { return Arrays.asList(recs); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java index 3f480596d62e2..f4aca9fa04aaf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java @@ -57,7 +57,12 @@ public StateStore getStore(final String name) { } @Override - public Map checkpointedOffsets() { + public Map checkpointed() { return null; } + + @Override + public void checkpoint(final Map offsets) { + + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 15b1d25694457..5c72fc9bff78c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -22,6 +22,8 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; @@ -39,6 +41,8 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockProcessorNode; import org.apache.kafka.test.MockSourceNode; @@ -415,6 +419,62 @@ restoreStateConsumer, createConfig(baseDir), streamsMetrics, assertTrue(flushed.get()); } + @SuppressWarnings("unchecked") + @Test + public void shouldCheckpointOffsetsOnCommit() throws Exception { + final String storeName = "test"; + final String changelogTopic = ProcessorStateManager.storeChangelogTopic("appId", storeName); + final InMemoryKeyValueStore inMemoryStore = new InMemoryKeyValueStore(storeName, null, null) { + @Override + public void init(final ProcessorContext context, final StateStore root) { + context.register(root, true, null); + } + + @Override + public boolean persistent() { + return true; + } + }; + final ProcessorTopology topology = new ProcessorTopology(Collections.emptyList(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.singletonList(inMemoryStore), + Collections.singletonMap(storeName, changelogTopic), + Collections.emptyList()); + + final TopicPartition partition = new TopicPartition(changelogTopic, 0); + final NoOpRecordCollector recordCollector = new NoOpRecordCollector() { + @Override + public Map offsets() { + + return Collections.singletonMap(partition, 543L); + } + }; + + restoreStateConsumer.updatePartitions(changelogTopic, + Collections.singletonList( + new PartitionInfo(changelogTopic, 0, null, new Node[0], new Node[0]))); + restoreStateConsumer.updateEndOffsets(Collections.singletonMap(partition, 0L)); + restoreStateConsumer.updateBeginningOffsets(Collections.singletonMap(partition, 0L)); + + final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics()); + final TaskId taskId = new TaskId(0, 0); + final MockTime time = new MockTime(); + final StreamsConfig config = createConfig(baseDir); + final StreamTask streamTask = new StreamTask(taskId, "appId", partitions, topology, consumer, + restoreStateConsumer, config, streamsMetrics, + stateDirectory, new ThreadCache("testCache", 0, streamsMetrics), + time, recordCollector); + + time.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)); + + streamTask.commit(); + final OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), + ProcessorStateManager.CHECKPOINT_FILE_NAME)); + + assertThat(checkpoint.read(), equalTo(Collections.singletonMap(partition, 544L))); + } + @Test public void shouldThrowIllegalStateExceptionIfCurrentNodeIsNotNullWhenPunctuateCalled() throws Exception { ((ProcessorContextImpl) task.processorContext()).setCurrentNode(processor); diff --git a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java index 2f3ef26635ecf..612a0da47e6d4 100644 --- a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java +++ b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java @@ -66,6 +66,11 @@ public void close(final Map offsets) throws IOException { closed = true; } + @Override + public void checkpoint(final Map offsets) { + this.offsets.putAll(offsets); + } + @Override public StateStore getGlobalStore(final String name) { return null; @@ -77,7 +82,7 @@ public StateStore getStore(final String name) { } @Override - public Map checkpointedOffsets() { + public Map checkpointed() { return offsets; } } diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index b50ff347b297e..ac8933dcb6aac 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -211,7 +211,8 @@ public List partitionsFor(String topic) { final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(globalTopology, globalConsumer, stateDirectory); globalStateTask = new GlobalStateUpdateTask(globalTopology, new GlobalProcessorContextImpl(config, stateManager, streamsMetrics, cache), - stateManager); + stateManager + ); globalStateTask.initialize(); } From a515e93570c5c117ca29fb05137898817f058fb9 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 17 Feb 2017 16:31:00 -0800 Subject: [PATCH 010/101] HOTFIX: ClassCastException in request logging Author: Jason Gustafson Reviewers: Colin P. Mccabe , Ewen Cheslack-Postava Closes #2566 from hachikuji/hotfix-request-logging --- core/src/main/scala/kafka/network/RequestChannel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index a5d71602eaeef..677f9b4fdbdc3 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -108,7 +108,7 @@ object RequestChannel extends Logging { if (requestObj != null) requestObj.describe(details) else - s"$header -- $body" + s"$header -- ${body[AbstractRequest]}" } def body[T <: AbstractRequest : ClassTag] = { From d734f4e56d276f84b8c52b602edd67d41cbb6c35 Mon Sep 17 00:00:00 2001 From: Prabhat Kashyap Date: Sat, 18 Feb 2017 06:22:08 -0800 Subject: [PATCH 011/101] MINOR: Replace for with foreach loop in common module Author: Prabhat Kashyap Reviewers: Ismael Juma Closes #2530 from PKOfficial/refactored-code --- .../apache/kafka/common/metrics/Sensor.java | 19 ++++++++---------- .../kafka/common/metrics/stats/Avg.java | 3 +-- .../kafka/common/metrics/stats/Count.java | 4 ++-- .../kafka/common/metrics/stats/Max.java | 4 ++-- .../kafka/common/metrics/stats/Min.java | 4 ++-- .../common/metrics/stats/Percentiles.java | 4 ++-- .../kafka/common/metrics/stats/Rate.java | 4 ++-- .../common/metrics/stats/SampledStat.java | 3 +-- .../common/requests/MetadataResponse.java | 20 ++++++++++--------- .../org/apache/kafka/common/utils/Utils.java | 6 +++--- 10 files changed, 34 insertions(+), 37 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index 4a9b488d1c5e0..e85b55915cb35 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -117,8 +117,8 @@ public boolean shouldRecord(final int configId) { private void checkForest(Set sensors) { if (!sensors.add(this)) throw new IllegalArgumentException("Circular dependency in sensors: " + name() + " is its own parent."); - for (int i = 0; i < parents.length; i++) - parents[i].checkForest(sensors); + for (Sensor parent : parents) + parent.checkForest(sensors); } /** @@ -168,12 +168,12 @@ public void record(double value, long timeMs) { this.lastRecordTime = timeMs; synchronized (this) { // increment all the stats - for (int i = 0; i < this.stats.size(); i++) - this.stats.get(i).record(config, value, timeMs); + for (Stat stat : this.stats) + stat.record(config, value, timeMs); checkQuotas(timeMs); } - for (int i = 0; i < parents.length; i++) - parents[i].record(value, timeMs); + for (Sensor parent : parents) + parent.record(value, timeMs); } } @@ -185,17 +185,14 @@ public void checkQuotas() { } public void checkQuotas(long timeMs) { - for (int i = 0; i < this.metrics.size(); i++) { - KafkaMetric metric = this.metrics.get(i); + for (KafkaMetric metric : this.metrics) { MetricConfig config = metric.config(); if (config != null) { Quota quota = config.quota(); if (quota != null) { double value = metric.value(timeMs); if (!quota.acceptable(value)) { - throw new QuotaViolationException( - metric.metricName(), - value, + throw new QuotaViolationException(metric.metricName(), value, quota.bound()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java index 0fe7380fe3762..a65dd287f4afd 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java @@ -34,8 +34,7 @@ protected void update(Sample sample, MetricConfig config, double value, long now public double combine(List samples, MetricConfig config, long now) { double total = 0.0; long count = 0; - for (int i = 0; i < samples.size(); i++) { - Sample s = samples.get(i); + for (Sample s : samples) { total += s.value; count += s.eventCount; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java index 90c0bf549a9ec..9ca3231ea77b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java @@ -33,8 +33,8 @@ protected void update(Sample sample, MetricConfig config, double value, long now @Override public double combine(List samples, MetricConfig config, long now) { double total = 0.0; - for (int i = 0; i < samples.size(); i++) - total += samples.get(i).value; + for (Sample sample : samples) + total += sample.value; return total; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java index 6bbb0a3a706c5..f0d741c965274 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java @@ -33,8 +33,8 @@ protected void update(Sample sample, MetricConfig config, double value, long now @Override public double combine(List samples, MetricConfig config, long now) { double max = Double.NEGATIVE_INFINITY; - for (int i = 0; i < samples.size(); i++) - max = Math.max(max, samples.get(i).value); + for (Sample sample : samples) + max = Math.max(max, sample.value); return max; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java index 113d745c3b185..5cfb535f460ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java @@ -33,8 +33,8 @@ protected void update(Sample sample, MetricConfig config, double value, long now @Override public double combine(List samples, MetricConfig config, long now) { double min = Double.MAX_VALUE; - for (int i = 0; i < samples.size(); i++) - min = Math.min(min, samples.get(i).value); + for (Sample sample : samples) + min = Math.min(min, sample.value); return min; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java index 12064f51f52fa..df0ee4a0288d4 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java @@ -78,8 +78,8 @@ public double value(MetricConfig config, long now, double quantile) { float sum = 0.0f; float quant = (float) quantile; for (int b = 0; b < buckets; b++) { - for (int s = 0; s < this.samples.size(); s++) { - HistogramSample sample = (HistogramSample) this.samples.get(s); + for (Sample s : this.samples) { + HistogramSample sample = (HistogramSample) s; float[] hist = sample.histogram.counts(); sum += hist[b]; if (sum / count > quant) diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index 6e496490bba86..c0dd10e6e459e 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -125,8 +125,8 @@ protected void update(Sample sample, MetricConfig config, double value, long tim @Override public double combine(List samples, MetricConfig config, long now) { double total = 0.0; - for (int i = 0; i < samples.size(); i++) - total += samples.get(i).value; + for (Sample sample : samples) + total += sample.value; return total; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java index 7d52ed339f6bf..58ea93f99c96a 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java @@ -96,8 +96,7 @@ public Sample oldest(long now) { /* Timeout any windows that have expired in the absence of any events */ protected void purgeObsoleteSamples(MetricConfig config, long now) { long expireAge = config.samples() * config.timeWindowMs(); - for (int i = 0; i < samples.size(); i++) { - Sample sample = this.samples.get(i); + for (Sample sample : samples) { if (now - sample.lastWindowMs >= expireAge) sample.reset(now); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 268bf8438a12c..1532f67d571ed 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -94,14 +94,14 @@ public MetadataResponse(List brokers, String clusterId, int controllerId, public MetadataResponse(Struct struct) { Map brokers = new HashMap<>(); Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME); - for (int i = 0; i < brokerStructs.length; i++) { - Struct broker = (Struct) brokerStructs[i]; + for (Object brokerStruct : brokerStructs) { + Struct broker = (Struct) brokerStruct; int nodeId = broker.getInt(NODE_ID_KEY_NAME); String host = broker.getString(HOST_KEY_NAME); int port = broker.getInt(PORT_KEY_NAME); // This field only exists in v1+ // When we can't know if a rack exists in a v0 response we default to null - String rack = broker.hasField(RACK_KEY_NAME) ? broker.getString(RACK_KEY_NAME) : null; + String rack = broker.hasField(RACK_KEY_NAME) ? broker.getString(RACK_KEY_NAME) : null; brokers.put(nodeId, new Node(nodeId, host, port, rack)); } @@ -120,8 +120,8 @@ public MetadataResponse(Struct struct) { List topicMetadata = new ArrayList<>(); Object[] topicInfos = (Object[]) struct.get(TOPIC_METADATA_KEY_NAME); - for (int i = 0; i < topicInfos.length; i++) { - Struct topicInfo = (Struct) topicInfos[i]; + for (Object topicInfoObj : topicInfos) { + Struct topicInfo = (Struct) topicInfoObj; Errors topicError = Errors.forCode(topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME)); String topic = topicInfo.getString(TOPIC_KEY_NAME); // This field only exists in v1+ @@ -131,8 +131,8 @@ public MetadataResponse(Struct struct) { List partitionMetadata = new ArrayList<>(); Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME); - for (int j = 0; j < partitionInfos.length; j++) { - Struct partitionInfo = (Struct) partitionInfos[j]; + for (Object partitionInfoObj : partitionInfos) { + Struct partitionInfo = (Struct) partitionInfoObj; Errors partitionError = Errors.forCode(partitionInfo.getShort(PARTITION_ERROR_CODE_KEY_NAME)); int partition = partitionInfo.getInt(PARTITION_KEY_NAME); int leader = partitionInfo.getInt(LEADER_KEY_NAME); @@ -140,19 +140,21 @@ public MetadataResponse(Struct struct) { Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); List replicaNodes = new ArrayList<>(replicas.length); - for (Object replicaNodeId : replicas) + for (Object replicaNodeId : replicas) { if (brokers.containsKey(replicaNodeId)) replicaNodes.add(brokers.get(replicaNodeId)); else replicaNodes.add(new Node((int) replicaNodeId, "", -1)); + } Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME); List isrNodes = new ArrayList<>(isr.length); - for (Object isrNode : isr) + for (Object isrNode : isr) { if (brokers.containsKey(isrNode)) isrNodes.add(brokers.get(isrNode)); else isrNodes.add(new Node((int) isrNode, "", -1)); + } partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes)); } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index ab89f6bc3e205..c8f57bb0e62bb 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -228,9 +228,9 @@ public static int abs(int n) { */ public static long min(long first, long ... rest) { long min = first; - for (int i = 0; i < rest.length; i++) { - if (rest[i] < min) - min = rest[i]; + for (long r : rest) { + if (r < min) + min = r; } return min; } From c6fcc721e2973f0bf2610d76bee4072bc30d48c9 Mon Sep 17 00:00:00 2001 From: Apurva Mehta Date: Sat, 18 Feb 2017 06:38:35 -0800 Subject: [PATCH 012/101] MINOR: Increase consumer init timeout in throttling test The throttling system test sometimes fail because it takes longer than the current 10 second time out for partitions to get assigned to the consumer. Author: Apurva Mehta Reviewers: Ismael Juma Closes #2567 from apurvam/increase-timeout-for-partitions-assigned --- tests/kafkatest/tests/core/throttling_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/kafkatest/tests/core/throttling_test.py b/tests/kafkatest/tests/core/throttling_test.py index 17a60ea5f7200..a80af39d3151f 100644 --- a/tests/kafkatest/tests/core/throttling_test.py +++ b/tests/kafkatest/tests/core/throttling_test.py @@ -53,7 +53,7 @@ def __init__(self, test_context): # ensure that the consumer is fully started before the producer starts # so that we don't miss any messages. This timeout ensures the sufficient # condition. - self.consumer_init_timeout_sec = 10 + self.consumer_init_timeout_sec = 20 self.num_brokers = 6 self.num_partitions = 3 self.kafka = KafkaService(test_context, From 916edc3a4ed0b44e64d0a0410137bf96e2a5558d Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Sat, 18 Feb 2017 06:40:35 -0800 Subject: [PATCH 013/101] =?UTF-8?q?KAFKA-4774;=20Inner=20classes=20which?= =?UTF-8?q?=20don't=20need=20a=20reference=20to=20the=20outer=20c=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …lass should be static Author: Colin P. Mccabe Reviewers: Jason Gustafson , Ismael Juma Closes #2558 from cmccabe/KAFKA-4774 --- .../java/org/apache/kafka/common/security/ssl/SslFactory.java | 2 +- .../connect/runtime/rest/resources/ConnectorsResource.java | 4 ++-- .../streams/processor/internals/GlobalStateUpdateTask.java | 2 +- .../processor/internals/assignment/StickyTaskAssignor.java | 2 +- .../org/apache/kafka/streams/state/internals/NamedCache.java | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java index 09af52070837e..ca49a3be43a37 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java @@ -196,7 +196,7 @@ private void createTruststore(String type, String path, Password password) { } } - private class SecurityStore { + private static class SecurityStore { private final String type; private final String path; private final Password password; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index e8f134d8349ff..366aac5bc432d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -285,14 +285,14 @@ private interface Translator { T translate(RestServer.HttpResponse response); } - private class IdentityTranslator implements Translator { + private static class IdentityTranslator implements Translator { @Override public T translate(RestServer.HttpResponse response) { return response.body(); } } - private class CreatedConnectorInfoTranslator implements Translator, ConnectorInfo> { + private static class CreatedConnectorInfoTranslator implements Translator, ConnectorInfo> { @Override public Herder.Created translate(RestServer.HttpResponse response) { boolean created = response.status() == 201; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index 6da37e4c1c003..a5504d1b453e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -29,7 +29,7 @@ */ public class GlobalStateUpdateTask implements GlobalStateMaintainer { - private class SourceNodeAndDeserializer { + private static class SourceNodeAndDeserializer { private final SourceNode sourceNode; private final RecordDeserializer deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java index 6d49b72d46560..7c81f6438f2cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java @@ -253,7 +253,7 @@ Pair pair(final TaskId task1, final TaskId task2) { return new Pair(task2, task1); } - class Pair { + private static class Pair { private final TaskId task1; private final TaskId task2; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index 931ab17eea6cf..8dcbf904668f3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -320,7 +320,7 @@ synchronized void close() { /** * A simple wrapper class to implement a doubly-linked list around MemoryLRUCacheBytesEntry */ - class LRUNode { + static class LRUNode { private final Bytes key; private LRUCacheEntry entry; private LRUNode previous; From dcea49856805a039f0859facf169a87a574c06d3 Mon Sep 17 00:00:00 2001 From: Allen Xiang Date: Sat, 18 Feb 2017 09:25:28 -0800 Subject: [PATCH 014/101] KAFKA-4777; Backoff properly in consumer heartbeat thread if no brokers are available Author: Allen Xiang Reviewers: Jason Gustafson Closes #2564 from allenxiang/client-heartbeat-fix --- .../clients/consumer/internals/AbstractCoordinator.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 350a84bf02268..1c2d607713498 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -888,9 +888,9 @@ public void run() { long now = time.milliseconds(); if (coordinatorUnknown()) { - if (findCoordinatorFuture == null) - lookupCoordinator(); - else + if (findCoordinatorFuture != null || lookupCoordinator().failed()) + // the immediate future check ensures that we backoff properly in the case that no + // brokers are available to connect to. AbstractCoordinator.this.wait(retryBackoffMs); } else if (heartbeat.sessionTimeoutExpired(now)) { // the session timeout has expired without seeing a successful heartbeat, so we should @@ -941,7 +941,7 @@ public void onFailure(RuntimeException e) { log.error("Unexpected interrupt received in heartbeat thread for group {}", groupId, e); this.failed.set(new RuntimeException(e)); } catch (RuntimeException e) { - log.error("Heartbeat thread for group {} failed due to unexpected error" , groupId, e); + log.error("Heartbeat thread for group {} failed due to unexpected error", groupId, e); this.failed.set(e); } finally { log.debug("Heartbeat thread for group {} has closed", groupId); From d24d932efb980b1fc15bb4dbba4d67d464e97ab3 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 18 Feb 2017 10:02:52 -0800 Subject: [PATCH 015/101] HOTFIX: ClassCastException in Request error logging Fixed ClassCastException resulting from missing type hint in request logging. Author: Armin Braun Reviewers: Jason Gustafson Closes #2571 from original-brownbear/fix-logging-err-response --- core/src/main/scala/kafka/server/KafkaApis.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c90cace9bcfcb..fa5afe5f68ba4 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -117,7 +117,7 @@ class KafkaApis(val requestChannel: RequestChannel, else requestChannel.sendResponse(new Response(request, response)) - error("Error when handling request %s".format(request.body), e) + error("Error when handling request %s".format(request.body[AbstractRequest]), e) } } finally request.apiLocalCompleteTimeMs = time.milliseconds From 3db3d73de6bc1fbb846d0943a8fe73dcca96b78a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 19 Feb 2017 16:39:36 -0800 Subject: [PATCH 016/101] MINOR: Fixed 3 inner classes without instance reference to be static * Turned 3 inner classes that weren't static but could be into `static` ones. * Turned one `public` inner class that wasn't used publicly into a `private`. Trivial but imo worthwhile to explicitly keep visibility and "staticness" correct in syntax (if only to be nice to the GC) :) Author: Armin Braun Reviewers: Matthias J. Sax , Jason Gustafson Closes #2574 from original-brownbear/cleanup-inner-nonstatic --- .../kafka/streams/kstream/internals/KStreamPassThrough.java | 4 ++-- .../streams/kstream/internals/KTableKTableJoinMerger.java | 3 ++- .../kafka/streams/processor/internals/ProcessorNode.java | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java index 59a815ba2b9b8..adf8e38e640bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java @@ -25,10 +25,10 @@ class KStreamPassThrough implements ProcessorSupplier { @Override public Processor get() { - return new KStreamPassThroughProcessor(); + return new KStreamPassThroughProcessor<>(); } - public class KStreamPassThroughProcessor extends AbstractProcessor { + private static final class KStreamPassThroughProcessor extends AbstractProcessor { @Override public void process(K key, V value) { context().forward(key, value); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java index f52e618466af7..558f4928d9c28 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java @@ -46,7 +46,8 @@ public void enableSendingOldValues() { parent2.enableSendingOldValues(); } - private class KTableKTableJoinMergeProcessor extends AbstractProcessor> { + private static final class KTableKTableJoinMergeProcessor + extends AbstractProcessor> { @Override public void process(K key, Change value) { context().forward(key, value); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index fa4cc2b96ebe5..41c5f7b405ef1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -110,7 +110,7 @@ public void addChild(ProcessorNode child) { public void init(ProcessorContext context) { this.context = context; try { - nodeMetrics = new NodeMetrics(context.metrics(), name, "task." + context.taskId()); + nodeMetrics = new NodeMetrics(context.metrics(), name, "task." + context.taskId()); nodeMetrics.metrics.measureLatencyNs(time, initDelegate, nodeMetrics.nodeCreationSensor); } catch (Exception e) { throw new StreamsException(String.format("failed to initialize processor %s", name), e); @@ -164,7 +164,7 @@ public String toString(String indent) { return sb.toString(); } - protected class NodeMetrics { + protected static final class NodeMetrics { final StreamsMetricsImpl metrics; final String metricGrpName; final Map metricTags; From e41e782006d807a1ca8098dbfb95b8ab2295d6af Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Sun, 19 Feb 2017 20:53:19 -0800 Subject: [PATCH 017/101] KAFKA-4776; Implement graceful handling for improperly formed compressed message sets Author: Jason Gustafson Reviewers: Jiangjie Qin Closes #2572 from hachikuji/KAFKA-4776 --- .../common/record/MemoryRecordsBuilder.java | 17 +++++--- .../apache/kafka/common/record/Record.java | 1 - .../kafka/common/record/RecordsIterator.java | 11 +++++- .../record/MemoryRecordsBuilderTest.java | 12 ++++++ .../kafka/common/record/SimpleRecordTest.java | 39 +++++++++++++++++++ 5 files changed, 71 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 39f21a99de496..731d1dc25778d 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -154,13 +154,18 @@ public void close() { throw new KafkaException(e); } - if (compressionType != CompressionType.NONE) - writerCompressedWrapperHeader(); + if (numRecords == 0L) { + buffer().position(initPos); + builtRecords = MemoryRecords.EMPTY; + } else { + if (compressionType != CompressionType.NONE) + writerCompressedWrapperHeader(); - ByteBuffer buffer = buffer().duplicate(); - buffer.flip(); - buffer.position(initPos); - builtRecords = MemoryRecords.readableRecords(buffer.slice()); + ByteBuffer buffer = buffer().duplicate(); + buffer.flip(); + buffer.position(initPos); + builtRecords = MemoryRecords.readableRecords(buffer.slice()); + } } private void writerCompressedWrapperHeader() { diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java index 51bbe35aa9411..e3d0b76ce9020 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Record.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java @@ -522,7 +522,6 @@ private static long write(DataOutputStream out, return crc; } - /** * Write a record using raw fields (without validation). This should only be used in testing. */ diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java index 07c919750c93e..430db30c6c8e2 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java @@ -146,8 +146,11 @@ public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, i this.wrapperMagic = wrapperRecord.magic(); CompressionType compressionType = wrapperRecord.compressionType(); - ByteBuffer buffer = wrapperRecord.value(); - DataInputStream stream = new DataInputStream(compressionType.wrapForInput(new ByteBufferInputStream(buffer), + ByteBuffer wrapperValue = wrapperRecord.value(); + if (wrapperValue == null) + throw new InvalidRecordException("Found invalid compressed record set with null value"); + + DataInputStream stream = new DataInputStream(compressionType.wrapForInput(new ByteBufferInputStream(wrapperValue), wrapperRecord.magic())); LogInputStream logStream = new DataLogInputStream(stream, maxMessageSize); @@ -180,6 +183,10 @@ public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, i } logEntries.addLast(logEntry); } + + if (logEntries.isEmpty()) + throw new InvalidRecordException("Found invalid compressed record set with no inner records"); + if (wrapperMagic > Record.MAGIC_VALUE_V0) this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset(); else diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java index 02ee75e00bcdd..4c2a71e040206 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java @@ -42,6 +42,18 @@ public MemoryRecordsBuilderTest(int bufferOffset, CompressionType compressionTyp this.compressionType = compressionType; } + @Test + public void testWriteEmptyRecordSet() { + ByteBuffer buffer = ByteBuffer.allocate(128); + buffer.position(bufferOffset); + + MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V0, compressionType, + TimestampType.CREATE_TIME, 0L, 0L, buffer.capacity()); + MemoryRecords records = builder.build(); + assertEquals(0, records.sizeInBytes()); + assertEquals(bufferOffset, buffer.position()); + } + @Test public void testCompressionRateV0() { ByteBuffer buffer = ByteBuffer.allocate(1024); diff --git a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java index e4c4a6781b9d4..ca78fc9cbd5a3 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java @@ -16,16 +16,55 @@ */ package org.apache.kafka.common.record; +import org.apache.kafka.common.utils.Utils; import org.junit.Test; +import java.io.DataOutputStream; +import java.io.OutputStream; import java.nio.ByteBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class SimpleRecordTest { + @Test(expected = InvalidRecordException.class) + public void testCompressedIterationWithNullValue() throws Exception { + ByteBuffer buffer = ByteBuffer.allocate(128); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buffer)); + LogEntry.writeHeader(out, 0L, Record.RECORD_OVERHEAD_V1); + Record.write(out, Record.CURRENT_MAGIC_VALUE, 1L, null, null, CompressionType.GZIP, TimestampType.CREATE_TIME); + + buffer.flip(); + + MemoryRecords records = MemoryRecords.readableRecords(buffer); + for (Record record : records.records()) + fail("Iteration should have caused invalid record error"); + } + + @Test(expected = InvalidRecordException.class) + public void testCompressedIterationWithEmptyRecords() throws Exception { + ByteBuffer emptyCompressedValue = ByteBuffer.allocate(64); + OutputStream gzipOutput = CompressionType.GZIP.wrapForOutput(new ByteBufferOutputStream(emptyCompressedValue), + Record.MAGIC_VALUE_V1, 64); + gzipOutput.close(); + emptyCompressedValue.flip(); + + ByteBuffer buffer = ByteBuffer.allocate(128); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buffer)); + LogEntry.writeHeader(out, 0L, Record.RECORD_OVERHEAD_V1 + emptyCompressedValue.remaining()); + Record.write(out, Record.CURRENT_MAGIC_VALUE, 1L, null, Utils.toArray(emptyCompressedValue), + CompressionType.GZIP, TimestampType.CREATE_TIME); + + buffer.flip(); + + MemoryRecords records = MemoryRecords.readableRecords(buffer); + for (Record record : records.records()) + fail("Iteration should have caused invalid record error"); + } + /* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */ @Test(expected = InvalidRecordException.class) public void testIsValidWithTooSmallBuffer() { From 3b8b7a4be39cb4a3f7257f8a17f74a887572e627 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 21 Feb 2017 13:52:18 +0000 Subject: [PATCH 018/101] MINOR: Fix NPE handling unknown APIs in NodeApiVersions.toString Author: Jason Gustafson Reviewers: Colin P. Mccabe , Ismael Juma Closes #2561 from hachikuji/fix-npe-api-version-tostring --- .../org/apache/kafka/clients/NodeApiVersions.java | 11 +++++------ .../org/apache/kafka/clients/NodeApiVersionsTest.java | 8 ++++++++ 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java index aa9c85a5664ce..906c2264c55ce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java @@ -31,7 +31,6 @@ * An internal class which represents the API versions supported by a particular node. */ public class NodeApiVersions { - private static final Short API_NOT_ON_NODE = null; private static final short NODE_TOO_OLD = (short) -1; private static final short NODE_TOO_NEW = (short) -2; private final Collection nodeApiVersions; @@ -47,7 +46,7 @@ public class NodeApiVersions { * @return A new NodeApiVersions object. */ public static NodeApiVersions create() { - return create(Collections.EMPTY_LIST); + return create(Collections.emptyList()); } /** @@ -98,7 +97,7 @@ public NodeApiVersions(Collection nodeApiVersions) { */ public short usableVersion(ApiKeys apiKey) { Short usableVersion = usableVersions.get(apiKey); - if (usableVersion == API_NOT_ON_NODE) + if (usableVersion == null) throw new UnsupportedVersionException("The broker does not support " + apiKey); else if (usableVersion == NODE_TOO_OLD) throw new UnsupportedVersionException("The broker is too old to support " + apiKey + @@ -160,17 +159,17 @@ private String apiVersionToText(ApiVersion apiVersion) { ApiKeys apiKey = null; if (ApiKeys.hasId(apiVersion.apiKey)) { apiKey = ApiKeys.forId(apiVersion.apiKey); - } - if (apiKey != null) { bld.append(apiKey.name).append("(").append(apiKey.id).append("): "); } else { - bld.append("UNKNOWN(").append(apiKey.id).append("): "); + bld.append("UNKNOWN(").append(apiVersion.apiKey).append("): "); } + if (apiVersion.minVersion == apiVersion.maxVersion) { bld.append(apiVersion.minVersion); } else { bld.append(apiVersion.minVersion).append(" to ").append(apiVersion.maxVersion); } + if (apiKey != null) { Short usableVersion = usableVersions.get(apiKey); if (usableVersion == NODE_TOO_OLD) diff --git a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java index 8773d9cd2a59b..53c47c833bfb7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java @@ -30,6 +30,7 @@ import java.util.List; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class NodeApiVersionsTest { @@ -47,6 +48,13 @@ public void testUnsupportedVersionsToString() { assertEquals(bld.toString(), versions.toString()); } + @Test + public void testUnknownApiVersionsToString() { + ApiVersion unknownApiVersion = new ApiVersion((short) 337, (short) 0, (short) 1); + NodeApiVersions versions = new NodeApiVersions(Collections.singleton(unknownApiVersion)); + assertTrue(versions.toString().endsWith("UNKNOWN(337): 0 to 1)")); + } + @Test public void testVersionsToString() { List versionList = new ArrayList<>(); From f48e8c1a88404c9ef386707c5ac79e21dc8df5af Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon" Date: Tue, 21 Feb 2017 13:59:25 +0000 Subject: [PATCH 019/101] MINOR: Add build_eclipse to .gitignore build_eclipse is the configured output directory for eclipse when using the gradle eclipse plugin and should be ignored Author: Christopher L. Shannon Reviewers: Ismael Juma Closes #2569 from cshannon/eclipse-gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index b3c646b8b35bf..b1439ccb6d8a1 100644 --- a/.gitignore +++ b/.gitignore @@ -2,6 +2,7 @@ dist *classes target/ build/ +build_eclipse/ .gradle/ lib_managed/ src_managed/ From 8bd06482d0562a518bce67dd06c590da4391d12a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 21 Feb 2017 14:02:17 +0000 Subject: [PATCH 020/101] MINOR: Remove unused MessageWriter and CompressionFactory Author: Jason Gustafson Reviewers: Apurva Mehta , Ismael Juma Closes #2543 from hachikuji/remove-message-writer --- .../common/record/CompressionTypeTest.java | 55 +++++ .../kafka/common/record/KafkaLZ4Test.java | 2 +- .../message/ByteBufferBackedInputStream.scala | 40 --- .../kafka/message/CompressionFactory.scala | 56 ----- .../scala/kafka/message/MessageWriter.scala | 229 ------------------ .../message/MessageCompressionTest.scala | 21 -- .../kafka/message/MessageWriterTest.scala | 131 ---------- 7 files changed, 56 insertions(+), 478 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java delete mode 100644 core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala delete mode 100644 core/src/main/scala/kafka/message/CompressionFactory.scala delete mode 100755 core/src/main/scala/kafka/message/MessageWriter.scala delete mode 100644 core/src/test/scala/unit/kafka/message/MessageWriterTest.scala diff --git a/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java new file mode 100644 index 0000000000000..68e2a63cc43a8 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ +package org.apache.kafka.common.record; + +import org.junit.Test; + +import java.nio.ByteBuffer; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class CompressionTypeTest { + + @Test + public void testLZ4FramingMagicV0() { + ByteBuffer buffer = ByteBuffer.allocate(256); + KafkaLZ4BlockOutputStream out = (KafkaLZ4BlockOutputStream) CompressionType.LZ4.wrapForOutput( + new ByteBufferOutputStream(buffer), Record.MAGIC_VALUE_V0, 256); + assertTrue(out.useBrokenFlagDescriptorChecksum()); + + buffer.rewind(); + + KafkaLZ4BlockInputStream in = (KafkaLZ4BlockInputStream) CompressionType.LZ4.wrapForInput( + new ByteBufferInputStream(buffer), Record.MAGIC_VALUE_V0); + assertTrue(in.ignoreFlagDescriptorChecksum()); + } + + @Test + public void testLZ4FramingMagicV1() { + ByteBuffer buffer = ByteBuffer.allocate(256); + KafkaLZ4BlockOutputStream out = (KafkaLZ4BlockOutputStream) CompressionType.LZ4.wrapForOutput( + new ByteBufferOutputStream(buffer), Record.MAGIC_VALUE_V1, 256); + assertFalse(out.useBrokenFlagDescriptorChecksum()); + + buffer.rewind(); + + KafkaLZ4BlockInputStream in = (KafkaLZ4BlockInputStream) CompressionType.LZ4.wrapForInput( + new ByteBufferInputStream(buffer), Record.MAGIC_VALUE_V1); + assertFalse(in.ignoreFlagDescriptorChecksum()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java b/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java index 47aebcbf5c683..aa50426d5c1ab 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java +++ b/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java @@ -54,7 +54,7 @@ public KafkaLZ4Test(boolean useBrokenFlagDescriptorChecksum, boolean ignoreFlagD public static Collection data() { byte[] payload = new byte[1000]; Arrays.fill(payload, (byte) 1); - List values = new ArrayList(); + List values = new ArrayList<>(); for (boolean broken : Arrays.asList(false, true)) for (boolean ignore : Arrays.asList(false, true)) for (boolean close : Arrays.asList(false, true)) diff --git a/core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala b/core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala deleted file mode 100644 index 73dfd34ec0f08..0000000000000 --- a/core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala +++ /dev/null @@ -1,40 +0,0 @@ -/** - * 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.message - -import java.io.InputStream -import java.nio.ByteBuffer - -class ByteBufferBackedInputStream(buffer:ByteBuffer) extends InputStream { - override def read():Int = { - if (buffer.hasRemaining) - buffer.get() & 0xFF - else - -1 - } - - override def read(bytes:Array[Byte], off:Int, len:Int):Int = { - if (buffer.hasRemaining) { - // Read only what's left - val realLen = math.min(len, buffer.remaining()) - buffer.get(bytes, off, realLen) - realLen - } else - -1 - } -} diff --git a/core/src/main/scala/kafka/message/CompressionFactory.scala b/core/src/main/scala/kafka/message/CompressionFactory.scala deleted file mode 100644 index e02ed633153fe..0000000000000 --- a/core/src/main/scala/kafka/message/CompressionFactory.scala +++ /dev/null @@ -1,56 +0,0 @@ -/** - * 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.message - -import java.io.OutputStream -import java.util.zip.GZIPOutputStream -import java.util.zip.GZIPInputStream -import java.io.InputStream - -import org.apache.kafka.common.record.{KafkaLZ4BlockInputStream, KafkaLZ4BlockOutputStream} - -object CompressionFactory { - - def apply(compressionCodec: CompressionCodec, messageVersion: Byte, stream: OutputStream): OutputStream = { - compressionCodec match { - case DefaultCompressionCodec => new GZIPOutputStream(stream) - case GZIPCompressionCodec => new GZIPOutputStream(stream) - case SnappyCompressionCodec => - import org.xerial.snappy.SnappyOutputStream - new SnappyOutputStream(stream) - case LZ4CompressionCodec => - new KafkaLZ4BlockOutputStream(stream, messageVersion == Message.MagicValue_V0) - case _ => - throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) - } - } - - def apply(compressionCodec: CompressionCodec, messageVersion: Byte, stream: InputStream): InputStream = { - compressionCodec match { - case DefaultCompressionCodec => new GZIPInputStream(stream) - case GZIPCompressionCodec => new GZIPInputStream(stream) - case SnappyCompressionCodec => - import org.xerial.snappy.SnappyInputStream - new SnappyInputStream(stream) - case LZ4CompressionCodec => - new KafkaLZ4BlockInputStream(stream, messageVersion == Message.MagicValue_V0) - case _ => - throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) - } - } -} diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala deleted file mode 100755 index e6954ff8dfd2b..0000000000000 --- a/core/src/main/scala/kafka/message/MessageWriter.scala +++ /dev/null @@ -1,229 +0,0 @@ -/** - * 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.message - -import java.io.{InputStream, OutputStream} -import java.nio.ByteBuffer - -import org.apache.kafka.common.record.TimestampType -import org.apache.kafka.common.utils.Crc32 - -class MessageWriter(segmentSize: Int) extends BufferingOutputStream(segmentSize) { - - import Message._ - - def write(key: Array[Byte] = null, - codec: CompressionCodec, - timestamp: Long, - timestampType: TimestampType, - magicValue: Byte)(writePayload: OutputStream => Unit): Unit = { - withCrc32Prefix { - // write magic value - write(magicValue) - // write attributes - var attributes: Byte = 0 - if (codec.codec > 0) - attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte - if (magicValue > MagicValue_V0) - attributes = timestampType.updateAttributes(attributes) - write(attributes) - // Write timestamp - if (magicValue > MagicValue_V0) - writeLong(timestamp) - // write the key - if (key == null) { - writeInt(-1) - } else { - writeInt(key.length) - write(key, 0, key.length) - } - // write the payload with length prefix - withLengthPrefix { - writePayload(this) - } - } - } - - private def writeInt(value: Int): Unit = { - write(value >>> 24) - write(value >>> 16) - write(value >>> 8) - write(value) - } - - private def writeInt(out: ReservedOutput, value: Int): Unit = { - out.write(value >>> 24) - out.write(value >>> 16) - out.write(value >>> 8) - out.write(value) - } - - private def writeLong(value: Long): Unit = { - write((value >>> 56).toInt) - write((value >>> 48).toInt) - write((value >>> 40).toInt) - write((value >>> 32).toInt) - write((value >>> 24).toInt) - write((value >>> 16).toInt) - write((value >>> 8).toInt) - write(value.toInt) - } - - private def withCrc32Prefix(writeData: => Unit): Unit = { - // get a writer for CRC value - val crcWriter = reserve(CrcLength) - // save current position - var seg = currentSegment - val offset = currentSegment.written - // write data - writeData - // compute CRC32 - val crc = new Crc32() - if (offset < seg.written) crc.update(seg.bytes, offset, seg.written - offset) - seg = seg.next - while (seg != null) { - if (seg.written > 0) crc.update(seg.bytes, 0, seg.written) - seg = seg.next - } - // write CRC32 - writeInt(crcWriter, crc.getValue().toInt) - } - - private def withLengthPrefix(writeData: => Unit): Unit = { - // get a writer for length value - val lengthWriter = reserve(ValueSizeLength) - // save current size - val oldSize = size - // write data - writeData - // write length value - writeInt(lengthWriter, size - oldSize) - } - -} - -/* - * OutputStream that buffers incoming data in segmented byte arrays - * This does not copy data when expanding its capacity - * It has a ability to - * - write data directly to ByteBuffer - * - copy data from an input stream to interval segmented arrays directly - * - hold a place holder for an unknown value that can be filled in later - */ -class BufferingOutputStream(segmentSize: Int) extends OutputStream { - - protected final class Segment(size: Int) { - val bytes = new Array[Byte](size) - var written = 0 - var next: Segment = null - def freeSpace: Int = bytes.length - written - } - - protected class ReservedOutput(seg: Segment, offset: Int, length: Int) extends OutputStream { - private[this] var cur = seg - private[this] var off = offset - private[this] var len = length - - override def write(value: Int) = { - if (len <= 0) throw new IndexOutOfBoundsException() - if (cur.bytes.length <= off) { - cur = cur.next - off = 0 - } - cur.bytes(off) = value.toByte - off += 1 - len -= 1 - } - } - - protected var currentSegment = new Segment(segmentSize) - private[this] val headSegment = currentSegment - private[this] var filled = 0 - - def size(): Int = filled + currentSegment.written - - override def write(b: Int): Unit = { - if (currentSegment.freeSpace <= 0) addSegment() - currentSegment.bytes(currentSegment.written) = b.toByte - currentSegment.written += 1 - } - - override def write(b: Array[Byte], off: Int, len: Int) { - if (off >= 0 && off <= b.length && len >= 0 && off + len <= b.length) { - var remaining = len - var offset = off - while (remaining > 0) { - if (currentSegment.freeSpace <= 0) addSegment() - - val amount = math.min(currentSegment.freeSpace, remaining) - System.arraycopy(b, offset, currentSegment.bytes, currentSegment.written, amount) - currentSegment.written += amount - offset += amount - remaining -= amount - } - } else { - throw new IndexOutOfBoundsException() - } - } - - def write(in: InputStream): Unit = { - var amount = 0 - while (amount >= 0) { - currentSegment.written += amount - if (currentSegment.freeSpace <= 0) addSegment() - amount = in.read(currentSegment.bytes, currentSegment.written, currentSegment.freeSpace) - } - } - - private def addSegment() = { - filled += currentSegment.written - val newSeg = new Segment(segmentSize) - currentSegment.next = newSeg - currentSegment = newSeg - } - - private def skip(len: Int): Unit = { - if (len >= 0) { - var remaining = len - while (remaining > 0) { - if (currentSegment.freeSpace <= 0) addSegment() - - val amount = math.min(currentSegment.freeSpace, remaining) - currentSegment.written += amount - remaining -= amount - } - } else { - throw new IndexOutOfBoundsException() - } - } - - def reserve(len: Int): ReservedOutput = { - val out = new ReservedOutput(currentSegment, currentSegment.written, len) - skip(len) - out - } - - def writeTo(buffer: ByteBuffer): Unit = { - var seg = headSegment - while (seg != null) { - buffer.put(seg.bytes, 0, seg.written) - seg = seg.next - } - } - -} diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala index 5d2c8fbe14067..b0913dbeb5cf4 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -17,9 +17,6 @@ package kafka.message -import org.apache.kafka.common.record._ - -import java.io.ByteArrayInputStream import java.io.ByteArrayOutputStream import scala.collection._ import org.scalatest.junit.JUnitSuite @@ -28,24 +25,6 @@ import org.junit.Assert._ class MessageCompressionTest extends JUnitSuite { - @Test - def testLZ4FramingV0() { - val output = CompressionFactory(LZ4CompressionCodec, Message.MagicValue_V0, new ByteArrayOutputStream()) - assertTrue(output.asInstanceOf[KafkaLZ4BlockOutputStream].useBrokenFlagDescriptorChecksum()) - - val input = CompressionFactory(LZ4CompressionCodec, Message.MagicValue_V0, new ByteArrayInputStream(Array[Byte](0x04, 0x22, 0x4D, 0x18, 0x60, 0x40, 0x1A))) - assertTrue(input.asInstanceOf[KafkaLZ4BlockInputStream].ignoreFlagDescriptorChecksum()) - } - - @Test - def testLZ4FramingV1() { - val output = CompressionFactory(LZ4CompressionCodec, Message.MagicValue_V1, new ByteArrayOutputStream()) - assertFalse(output.asInstanceOf[KafkaLZ4BlockOutputStream].useBrokenFlagDescriptorChecksum()) - - val input = CompressionFactory(LZ4CompressionCodec, Message.MagicValue_V1, new ByteArrayInputStream(Array[Byte](0x04, 0x22, 0x4D, 0x18, 0x60, 0x40, -126))) - assertFalse(input.asInstanceOf[KafkaLZ4BlockInputStream].ignoreFlagDescriptorChecksum()) - } - @Test def testSimpleCompressDecompress() { val codecs = mutable.ArrayBuffer[CompressionCodec](GZIPCompressionCodec) diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala deleted file mode 100644 index a82a553370e09..0000000000000 --- a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala +++ /dev/null @@ -1,131 +0,0 @@ -/** - * 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.message - -import java.io.{InputStream, ByteArrayInputStream, ByteArrayOutputStream} -import java.nio.ByteBuffer -import java.util.Random -import org.apache.kafka.common.record.TimestampType -import org.junit.Assert._ -import org.junit.Test -import org.scalatest.junit.JUnitSuite - -class MessageWriterTest extends JUnitSuite { - - private val rnd = new Random() - - private def mkRandomArray(size: Int): Array[Byte] = { - (0 until size).map(_ => rnd.nextInt(10).toByte).toArray - } - - private def mkMessageWithWriter(key: Array[Byte] = null, bytes: Array[Byte], codec: CompressionCodec): Message = { - val writer = new MessageWriter(100) - writer.write(key = key, codec = codec, timestamp = Message.NoTimestamp, timestampType = TimestampType.CREATE_TIME, magicValue = Message.MagicValue_V1) { output => - val out = if (codec == NoCompressionCodec) output else CompressionFactory(codec, Message.MagicValue_V1, output) - try { - val p = rnd.nextInt(bytes.length) - out.write(bytes, 0, p) - out.write(bytes, p, bytes.length - p) - } finally { - out.close() - } - } - val bb = ByteBuffer.allocate(writer.size) - writer.writeTo(bb) - bb.rewind() - new Message(bb) - } - - private def compress(bytes: Array[Byte], codec: CompressionCodec): Array[Byte] = { - val baos = new ByteArrayOutputStream() - val out = CompressionFactory(codec, Message.MagicValue_V1, baos) - out.write(bytes) - out.close() - baos.toByteArray - } - - private def decompress(compressed: Array[Byte], codec: CompressionCodec): Array[Byte] = { - toArray(CompressionFactory(codec, Message.MagicValue_V1, new ByteArrayInputStream(compressed))) - } - - private def toArray(in: InputStream): Array[Byte] = { - val out = new ByteArrayOutputStream() - val buf = new Array[Byte](100) - var amount = in.read(buf) - while (amount >= 0) { - out.write(buf, 0, amount) - amount = in.read(buf) - } - out.toByteArray - } - - private def toArray(bb: ByteBuffer): Array[Byte] = { - val arr = new Array[Byte](bb.limit()) - bb.get(arr) - bb.rewind() - arr - } - - @Test - def testBufferingOutputStream(): Unit = { - val out = new BufferingOutputStream(50) - out.write(0) - out.write(1) - out.write(2) - val r = out.reserve(100) - out.write((103 until 200).map(_.toByte).toArray) - r.write((3 until 103).map(_.toByte).toArray) - - val buf = ByteBuffer.allocate(out.size) - out.writeTo(buf) - buf.rewind() - - assertEquals((0 until 200).map(_.toByte), buf.array.toSeq) - } - - @Test - def testWithNoCompressionAttribute(): Unit = { - val bytes = mkRandomArray(4096) - val actual = mkMessageWithWriter(bytes = bytes, codec = NoCompressionCodec) - val expected = new Message(bytes, Message.NoTimestamp, NoCompressionCodec, Message.MagicValue_V1) - assertEquals(expected.buffer, actual.buffer) - } - - @Test - def testWithCompressionAttribute(): Unit = { - val bytes = mkRandomArray(4096) - val actual = mkMessageWithWriter(bytes = bytes, codec = SnappyCompressionCodec) - val expected = new Message(compress(bytes, SnappyCompressionCodec), Message.NoTimestamp, SnappyCompressionCodec, Message.MagicValue_V1) - - assertEquals( - decompress(toArray(expected.payload), SnappyCompressionCodec).toSeq, - decompress(toArray(actual.payload), SnappyCompressionCodec).toSeq - ) - } - - @Test - def testWithKey(): Unit = { - val key = mkRandomArray(123) - val bytes = mkRandomArray(4096) - val actual = mkMessageWithWriter(bytes = bytes, key = key, codec = NoCompressionCodec) - val expected = new Message(bytes = bytes, key = key, timestamp = Message.NoTimestamp, codec = NoCompressionCodec, magicValue = Message.MagicValue_V1) - - assertEquals(expected.buffer, actual.buffer) - } - -} From 7b302add35762139c265b7658add144cd12c8278 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Tue, 21 Feb 2017 09:25:06 -0800 Subject: [PATCH 021/101] KAFKA-4752: Fixed bandwidth calculation Author: Eno Thereska Reviewers: Damian Guy, Guozhang Wang Closes #2551 from enothereska/KAFKA-4752-join-bw --- .../kafka/streams/perf/SimpleBenchmark.java | 43 ++++++++++++------- 1 file changed, 28 insertions(+), 15 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index 21efe50cbbcd1..4f41e1746d620 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -103,9 +103,8 @@ public byte[] apply(final byte[] value1, final byte[] value2) { private static int numRecords; private static int processedRecords = 0; - private static final int KEY_SIZE = 8; + private static long processedBytes = 0; private static final int VALUE_SIZE = 100; - private static final int RECORD_SIZE = KEY_SIZE + VALUE_SIZE; private static final Serde BYTE_SERDE = Serdes.ByteArray(); private static final Serde INTEGER_SERDE = Serdes.Integer(); @@ -229,6 +228,7 @@ private Properties setProduceConsumeProperties(final String clientId) { private boolean maybeSetupPhase(final String topic, final String clientId, final boolean skipIfAllTests) throws Exception { processedRecords = 0; + processedBytes = 0; // initialize topics if (loadPhase) { if (skipIfAllTests) { @@ -339,7 +339,7 @@ private void printResults(final String nameOfBenchmark, final long latency) { processedRecords + "/" + latency + "/" + recordsPerSec(latency, processedRecords) + "/" + - megabytesPerSec(latency, processedRecords, RECORD_SIZE)); + megabytesPerSec(latency, processedBytes)); } private void runGenericBenchmark(final KafkaStreams streams, final String nameOfBenchmark, final CountDownLatch latch) { @@ -475,6 +475,10 @@ private void produce(String topic, int valueSizeBytes, String clientId, int numR KafkaProducer producer = new KafkaProducer<>(props); byte[] value = new byte[valueSizeBytes]; + // put some random values to increase entropy. Some devices + // like SSDs do compression and if the array is all zeros + // the performance will be too good. + new Random().nextBytes(value); long startTime = System.currentTimeMillis(); if (sequential) key = 0; @@ -493,12 +497,11 @@ private void produce(String topic, int valueSizeBytes, String clientId, int numR numRecords + "/" + (endTime - startTime) + "/" + recordsPerSec(endTime - startTime, numRecords) + "/" + - megabytesPerSec(endTime - startTime, numRecords, KEY_SIZE + valueSizeBytes)); + megabytesPerSec(endTime - startTime, numRecords * valueSizeBytes)); } } public void consume(String topic) throws Exception { - int consumedRecords = 0; if (maybeSetupPhase(topic, "simple-benchmark-consumer-load", true)) { return; } @@ -518,19 +521,20 @@ public void consume(String topic) throws Exception { while (true) { ConsumerRecords records = consumer.poll(500); if (records.isEmpty()) { - if (consumedRecords == numRecords) + if (processedRecords == numRecords) break; } else { for (ConsumerRecord record : records) { - consumedRecords++; + processedRecords++; + processedBytes += record.value().length + Integer.SIZE; Integer recKey = record.key(); if (key == null || key < recKey) key = recKey; - if (consumedRecords == numRecords) + if (processedRecords == numRecords) break; } } - if (consumedRecords == numRecords) + if (processedRecords == numRecords) break; } @@ -538,10 +542,10 @@ public void consume(String topic) throws Exception { consumer.close(); System.out.println("Consumer Performance [records/latency/rec-sec/MB-sec read]: " + - consumedRecords + "/" + + processedRecords + "/" + (endTime - startTime) + "/" + - recordsPerSec(endTime - startTime, consumedRecords) + "/" + - megabytesPerSec(endTime - startTime, consumedRecords, RECORD_SIZE)); + recordsPerSec(endTime - startTime, processedRecords) + "/" + + megabytesPerSec(endTime - startTime, processedBytes)); } private KafkaStreams createKafkaStreams(String topic, final CountDownLatch latch) { @@ -563,6 +567,7 @@ public void init(ProcessorContext context) { @Override public void process(Integer key, byte[] value) { processedRecords++; + processedBytes += value.length + Integer.SIZE; if (processedRecords == numRecords) { latch.countDown(); } @@ -601,6 +606,7 @@ public void init(ProcessorContext context) { @Override public void process(Integer key, byte[] value) { processedRecords++; + processedBytes += value.length + Integer.SIZE; if (processedRecords == numRecords) { latch.countDown(); } @@ -628,6 +634,13 @@ private class CountDownAction implements ForeachAction { @Override public void apply(Integer key, V value) { processedRecords++; + if (value instanceof byte[]) { + processedBytes += ((byte[]) value).length + Integer.SIZE; + } else if (value instanceof Long) { + processedBytes += Long.SIZE + Integer.SIZE; + } else { + System.err.println("Unknown value type in CountDownAction"); + } if (processedRecords == numRecords) { this.latch.countDown(); } @@ -701,6 +714,7 @@ public void init(ProcessorContext context) { public void process(Integer key, byte[] value) { store.put(key, value); processedRecords++; + processedBytes += value.length + Integer.SIZE; if (processedRecords == numRecords) { latch.countDown(); } @@ -720,9 +734,8 @@ public void close() { return new KafkaStreams(builder, props); } - - private double megabytesPerSec(long time, int numRecords, int recordSizeBytes) { - return ((double) recordSizeBytes * numRecords / 1024 / 1024) / (time / 1000.0); + private double megabytesPerSec(long time, long processedBytes) { + return ((double) processedBytes / 1024 / 1024) / (time / 1000.0); } private double recordsPerSec(long time, int numRecords) { From 696b1d311c89671cdebd6e8ade4d2c2e55a0696c Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Tue, 21 Feb 2017 20:41:55 -0800 Subject: [PATCH 022/101] KAFKA-4757; NetworkClient should log request details at trace level when a request is cancelled because of disconnection Author: Colin P. Mccabe Reviewers: Onur Karaman , Apurva Mehta , Jiangjie Qin Closes #2563 from cmccabe/KAFKA-4757 --- .../java/org/apache/kafka/clients/NetworkClient.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 4131bcb41d9a3..a927910bebe61 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -316,10 +316,10 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long if (log.isDebugEnabled()) { int latestClientVersion = ProtoUtils.latestVersion(clientRequest.apiKey().id); if (header.apiVersion() == latestClientVersion) { - log.trace("Sending {} to node {}.", request, nodeId); + log.trace("Sending {} {} to node {}.", ApiKeys.forId(header.apiKey()), request, nodeId); } else { - log.debug("Using older server API v{} to send {} to node {}.", - header.apiVersion(), request, nodeId); + log.debug("Using older server API v{} to send {} {} to node {}.", + header.apiVersion(), ApiKeys.forId(header.apiKey()), request, nodeId); } } Send send = request.toSend(nodeId, header); @@ -330,6 +330,7 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long clientRequest.callback(), clientRequest.expectResponse(), isInternalRequest, + request, send, now); this.inFlightRequests.add(inFlightRequest); @@ -472,7 +473,7 @@ private void processDisconnection(List responses, String nodeId, nodeApiVersions.remove(nodeId); nodesNeedingApiVersionsFetch.remove(nodeId); for (InFlightRequest request : this.inFlightRequests.clearAll(nodeId)) { - log.trace("Cancelled request {} due to node {} being disconnected", request, nodeId); + log.trace("Cancelled request {} due to node {} being disconnected", request.request, nodeId); if (request.isInternalRequest && request.header.apiKey() == ApiKeys.METADATA.id) metadataUpdater.handleDisconnection(request.destination); else @@ -798,6 +799,7 @@ static class InFlightRequest { final String destination; final RequestCompletionHandler callback; final boolean expectResponse; + final AbstractRequest request; final boolean isInternalRequest; // used to flag requests which are initiated internally by NetworkClient final Send send; final long sendTimeMs; @@ -809,6 +811,7 @@ public InFlightRequest(RequestHeader header, RequestCompletionHandler callback, boolean expectResponse, boolean isInternalRequest, + AbstractRequest request, Send send, long sendTimeMs) { this.header = header; @@ -816,6 +819,7 @@ public InFlightRequest(RequestHeader header, this.callback = callback; this.expectResponse = expectResponse; this.isInternalRequest = isInternalRequest; + this.request = request; this.send = send; this.sendTimeMs = sendTimeMs; this.createdTimeMs = createdTimeMs; From 913c09e4a94447a953cb55ccd702d7131d1764a8 Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Wed, 22 Feb 2017 11:00:45 +0000 Subject: [PATCH 023/101] KAFKA-4708; Fix transient failure in BrokerApiVersionsCommandTest.checkBrokerApiVersionCommandOutput Author: Colin P. Mccabe Reviewers: Jason Gustafson , Dong Lin , Ismael Juma Closes #2489 from cmccabe/KAFKA-4708 --- core/src/main/scala/kafka/admin/AdminClient.scala | 14 +++++++++++++- .../kafka/admin/BrokerApiVersionsCommand.scala | 4 +++- .../kafka/admin/BrokerApiVersionsCommandTest.scala | 2 +- 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 25e64ebe55584..49c328e3e7ef1 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -85,7 +85,19 @@ class AdminClient(val time: Time, response.apiVersions.asScala.toList } - private def findAllBrokers(): List[Node] = { + /** + * Wait until there is a non-empty list of brokers in the cluster. + */ + def awaitBrokers() { + var nodes = List[Node]() + do { + nodes = findAllBrokers() + if (nodes.isEmpty) + Thread.sleep(50) + } while (nodes.isEmpty) + } + + def findAllBrokers(): List[Node] = { val request = MetadataRequest.Builder.allTopics() val response = sendAnyNode(ApiKeys.METADATA, request).asInstanceOf[MetadataResponse] val errors = response.errors diff --git a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala index 812bc9ddad77b..ac94a7e72e7f8 100644 --- a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala +++ b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala @@ -24,6 +24,7 @@ import kafka.utils.CommandLineUtils import org.apache.kafka.common.utils.Utils import org.apache.kafka.clients.CommonClientConfigs import joptsimple._ +import org.apache.kafka.common.Node import scala.util.{Failure, Success} @@ -39,7 +40,8 @@ object BrokerApiVersionsCommand { def execute(args: Array[String], out: PrintStream): Unit = { val opts = new BrokerVersionCommandOptions(args) val adminClient = createAdminClient(opts) - val brokerMap = adminClient.listAllBrokerVersionInfo() + adminClient.awaitBrokers() + var brokerMap = adminClient.listAllBrokerVersionInfo() brokerMap.foreach { case (broker, versionInfoOrError) => versionInfoOrError match { case Success(v) => out.print(s"${broker} -> ${v.toString(true)}\n") diff --git a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala index 8fa489d3fa331..cb369648863d6 100644 --- a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala +++ b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala @@ -33,7 +33,7 @@ class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { def generateConfigs(): Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps) - @Test + @Test(timeout=120000) def checkBrokerApiVersionCommandOutput() { val byteArrayOutputStream = new ByteArrayOutputStream val printStream = new PrintStream(byteArrayOutputStream) From 5916ef0227d099e5fa05341db3f918f5ef035816 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Wed, 22 Feb 2017 12:08:09 -0800 Subject: [PATCH 024/101] KAFKA-4786; Wait for heartbeat thread to terminate in consumer close Author: Rajini Sivaram Reviewers: Apurva Mehta , Ismael Juma , Jason Gustafson Closes #2586 from rajinisivaram/KAFKA-4786 --- .../internals/AbstractCoordinator.java | 46 +++++++++++++------ 1 file changed, 33 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 1c2d607713498..d36aac9f4e986 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -315,6 +315,19 @@ private synchronized void disableHeartbeatThread() { heartbeatThread.disable(); } + private void closeHeartbeatThread() { + if (heartbeatThread != null) { + heartbeatThread.close(); + + try { + heartbeatThread.join(); + } catch (InterruptedException e) { + log.warn("Interrupted while waiting for consumer heartbeat thread to close"); + throw new InterruptException(e); + } + } + } + // visible for testing. Joins the group without starting the heartbeat thread. void joinGroupIfNeeded() { while (needRejoin() || rejoinIncomplete()) { @@ -652,19 +665,26 @@ public synchronized void close() { close(0); } - protected synchronized void close(long timeoutMs) { - if (heartbeatThread != null) - heartbeatThread.close(); - maybeLeaveGroup(); - - // At this point, there may be pending commits (async commits or sync commits that were - // interrupted using wakeup) and the leave group request which have been queued, but not - // yet sent to the broker. Wait up to close timeout for these pending requests to be processed. - // If coordinator is not known, requests are aborted. - Node coordinator = coordinator(); - if (coordinator != null && !client.awaitPendingRequests(coordinator, timeoutMs)) - log.warn("Close timed out with {} pending requests to coordinator, terminating client connections for group {}.", - client.pendingRequestCount(coordinator), groupId); + protected void close(long timeoutMs) { + try { + closeHeartbeatThread(); + } finally { + + // Synchronize after closing the heartbeat thread since heartbeat thread + // needs this lock to complete and terminate after close flag is set. + synchronized (this) { + maybeLeaveGroup(); + + // At this point, there may be pending commits (async commits or sync commits that were + // interrupted using wakeup) and the leave group request which have been queued, but not + // yet sent to the broker. Wait up to close timeout for these pending requests to be processed. + // If coordinator is not known, requests are aborted. + Node coordinator = coordinator(); + if (coordinator != null && !client.awaitPendingRequests(coordinator, timeoutMs)) + log.warn("Close timed out with {} pending requests to coordinator, terminating client connections for group {}.", + client.pendingRequestCount(coordinator), groupId); + } + } } /** From 015f1d73811e52b3b7a578cf4c1ca4c48b6c2f5c Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 22 Feb 2017 15:11:12 -0800 Subject: [PATCH 025/101] MINOR: Move ProtoUtils methods to ApiKeys Also move `requireTimestamp` to `minVersion` logic from `Fetcher` to `ListOffsetRequest.Builder.forConsumer()`. Author: Ismael Juma Reviewers: Colin P. Mccabe , Jason Gustafson Closes #2580 from ijuma/move-proto-utils-to-api-keys --- .../apache/kafka/clients/ClientRequest.java | 2 +- .../apache/kafka/clients/NetworkClient.java | 12 ++- .../apache/kafka/clients/NodeApiVersions.java | 22 +++--- .../clients/consumer/internals/Fetcher.java | 4 +- .../apache/kafka/common/protocol/ApiKeys.java | 47 ++++++++++++ .../kafka/common/protocol/ProtoUtils.java | 73 ------------------- .../common/requests/AbstractRequest.java | 5 +- .../common/requests/AbstractResponse.java | 3 +- .../common/requests/ApiVersionsRequest.java | 7 +- .../common/requests/ApiVersionsResponse.java | 11 ++- .../requests/ControlledShutdownRequest.java | 9 +-- .../requests/ControlledShutdownResponse.java | 5 +- .../common/requests/CreateTopicsRequest.java | 9 +-- .../common/requests/CreateTopicsResponse.java | 5 +- .../common/requests/DeleteTopicsRequest.java | 9 +-- .../common/requests/DeleteTopicsResponse.java | 5 +- .../requests/DescribeGroupsRequest.java | 13 ++-- .../requests/DescribeGroupsResponse.java | 7 +- .../kafka/common/requests/FetchRequest.java | 7 +- .../kafka/common/requests/FetchResponse.java | 5 +- .../requests/GroupCoordinatorRequest.java | 10 +-- .../requests/GroupCoordinatorResponse.java | 5 +- .../common/requests/HeartbeatRequest.java | 9 +-- .../common/requests/HeartbeatResponse.java | 5 +- .../common/requests/JoinGroupRequest.java | 7 +- .../common/requests/JoinGroupResponse.java | 5 +- .../common/requests/LeaderAndIsrRequest.java | 9 +-- .../common/requests/LeaderAndIsrResponse.java | 5 +- .../common/requests/LeaveGroupRequest.java | 9 +-- .../common/requests/LeaveGroupResponse.java | 5 +- .../common/requests/ListGroupsRequest.java | 9 +-- .../common/requests/ListGroupsResponse.java | 7 +- .../common/requests/ListOffsetRequest.java | 13 ++-- .../common/requests/ListOffsetResponse.java | 5 +- .../common/requests/MetadataRequest.java | 9 +-- .../common/requests/MetadataResponse.java | 5 +- .../common/requests/OffsetCommitRequest.java | 11 ++- .../common/requests/OffsetCommitResponse.java | 5 +- .../common/requests/OffsetFetchRequest.java | 9 +-- .../common/requests/OffsetFetchResponse.java | 5 +- .../kafka/common/requests/ProduceRequest.java | 9 +-- .../common/requests/ProduceResponse.java | 5 +- .../common/requests/SaslHandshakeRequest.java | 11 ++- .../requests/SaslHandshakeResponse.java | 5 +- .../common/requests/StopReplicaRequest.java | 9 +-- .../common/requests/StopReplicaResponse.java | 7 +- .../common/requests/SyncGroupRequest.java | 9 +-- .../common/requests/SyncGroupResponse.java | 5 +- .../requests/UpdateMetadataRequest.java | 10 +-- .../requests/UpdateMetadataResponse.java | 5 +- .../kafka/clients/NetworkClientTest.java | 3 +- .../kafka/clients/NodeApiVersionsTest.java | 18 ++--- .../kafka/common/protocol/ApiKeysTest.java | 5 ++ .../kafka/common/protocol/ProtoUtilsTest.java | 26 ------- .../common/requests/RequestResponseTest.java | 23 +++--- .../authenticator/SaslAuthenticatorTest.java | 5 +- .../kafka/api/AuthorizerIntegrationTest.scala | 14 ++-- .../AbstractCreateTopicsRequestTest.scala | 5 +- .../unit/kafka/server/ApiVersionsTest.scala | 6 +- .../unit/kafka/server/FetchRequestTest.scala | 4 +- .../unit/kafka/server/MetadataCacheTest.scala | 14 ++-- 61 files changed, 253 insertions(+), 352 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java delete mode 100644 clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index a1973ad0edefd..564b391d2d7e4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -75,7 +75,7 @@ public ApiKeys apiKey() { } public RequestHeader makeHeader(short version) { - return new RequestHeader(requestBuilder.apiKey().id, version, clientId, correlationId); + return new RequestHeader(apiKey().id, version, clientId, correlationId); } public AbstractRequest.Builder requestBuilder() { diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index a927910bebe61..79d467af0212e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; @@ -314,12 +313,12 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long String nodeId = clientRequest.destination(); RequestHeader header = clientRequest.makeHeader(request.version()); if (log.isDebugEnabled()) { - int latestClientVersion = ProtoUtils.latestVersion(clientRequest.apiKey().id); + int latestClientVersion = clientRequest.apiKey().latestVersion(); if (header.apiVersion() == latestClientVersion) { - log.trace("Sending {} {} to node {}.", ApiKeys.forId(header.apiKey()), request, nodeId); + log.trace("Sending {} {} to node {}.", clientRequest.apiKey(), request, nodeId); } else { log.debug("Using older server API v{} to send {} {} to node {}.", - header.apiVersion(), ApiKeys.forId(header.apiKey()), request, nodeId); + header.apiVersion(), clientRequest.apiKey(), request, nodeId); } } Send send = request.toSend(nodeId, header); @@ -454,9 +453,8 @@ public Node leastLoadedNode(long now) { public static AbstractResponse parseResponse(ByteBuffer responseBuffer, RequestHeader requestHeader) { ResponseHeader responseHeader = ResponseHeader.parse(responseBuffer); // Always expect the response version id to be the same as the request version id - short apiKey = requestHeader.apiKey(); - short apiVer = requestHeader.apiVersion(); - Struct responseBody = ProtoUtils.responseSchema(apiKey, apiVer).read(responseBuffer); + ApiKeys apiKey = ApiKeys.forId(requestHeader.apiKey()); + Struct responseBody = apiKey.responseSchema(requestHeader.apiVersion()).read(responseBuffer); correlate(requestHeader, responseHeader); return AbstractResponse.getResponse(apiKey, responseBody); } diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java index 906c2264c55ce..0fcdf4d39bb4c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion; import org.apache.kafka.common.utils.Utils; @@ -67,8 +66,7 @@ public static NodeApiVersions create(Collection overrides) { } } if (!exists) { - apiVersions.add(new ApiVersion(apiKey.id, ProtoUtils.oldestVersion(apiKey.id), - ProtoUtils.latestVersion(apiKey.id))); + apiVersions.add(new ApiVersion(apiKey)); } } return new NodeApiVersions(apiVersions); @@ -77,16 +75,16 @@ public static NodeApiVersions create(Collection overrides) { public NodeApiVersions(Collection nodeApiVersions) { this.nodeApiVersions = nodeApiVersions; for (ApiVersion nodeApiVersion : nodeApiVersions) { - int nodeApiKey = nodeApiVersion.apiKey; // Newer brokers may support ApiKeys we don't know about, ignore them - if (ApiKeys.hasId(nodeApiKey)) { - short v = Utils.min(ProtoUtils.latestVersion(nodeApiKey), nodeApiVersion.maxVersion); + if (ApiKeys.hasId(nodeApiVersion.apiKey)) { + ApiKeys nodeApiKey = ApiKeys.forId(nodeApiVersion.apiKey); + short v = Utils.min(nodeApiKey.latestVersion(), nodeApiVersion.maxVersion); if (v < nodeApiVersion.minVersion) { - usableVersions.put(ApiKeys.forId(nodeApiKey), NODE_TOO_NEW); - } else if (v < ProtoUtils.oldestVersion(nodeApiKey)) { - usableVersions.put(ApiKeys.forId(nodeApiKey), NODE_TOO_OLD); + usableVersions.put(nodeApiKey, NODE_TOO_NEW); + } else if (v < nodeApiKey.oldestVersion()) { + usableVersions.put(nodeApiKey, NODE_TOO_OLD); } else { - usableVersions.put(ApiKeys.forId(nodeApiKey), v); + usableVersions.put(nodeApiKey, v); } } } @@ -101,10 +99,10 @@ public short usableVersion(ApiKeys apiKey) { throw new UnsupportedVersionException("The broker does not support " + apiKey); else if (usableVersion == NODE_TOO_OLD) throw new UnsupportedVersionException("The broker is too old to support " + apiKey + - " version " + ProtoUtils.oldestVersion(apiKey.id)); + " version " + apiKey.oldestVersion()); else if (usableVersion == NODE_TOO_NEW) throw new UnsupportedVersionException("The broker is too new to support " + apiKey + - " version " + ProtoUtils.latestVersion(apiKey.id)); + " version " + apiKey.latestVersion()); else return usableVersion; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 655c27bb260cd..d67b9402a25db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -605,9 +605,7 @@ public void onFailure(RuntimeException e) { private RequestFuture> sendListOffsetRequest(final Node node, final Map timestampsToSearch, boolean requireTimestamp) { - // If we need a timestamp in the response, the minimum RPC version we can send is v1. Otherwise, v0 is OK. - short minVersion = requireTimestamp ? (short) 1 : (short) 0; - ListOffsetRequest.Builder builder = ListOffsetRequest.Builder.forConsumer(minVersion) + ListOffsetRequest.Builder builder = ListOffsetRequest.Builder.forConsumer(requireTimestamp) .setTargetTimes(timestampsToSearch); log.trace("Sending ListOffsetRequest {} to broker {}", builder, node); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 239780ca72077..50d89f1e9f9c7 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -16,6 +16,11 @@ */ package org.apache.kafka.common.protocol; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; + /** * Identifiers for all the Kafka APIs */ @@ -64,6 +69,8 @@ public enum ApiKeys { public final String name; ApiKeys(int id, String name) { + if (id < 0) + throw new IllegalArgumentException("id must not be negative, id: " + id); this.id = (short) id; this.name = name; } @@ -79,6 +86,46 @@ public static boolean hasId(int id) { return id >= MIN_API_KEY && id <= MAX_API_KEY; } + public short latestVersion() { + if (id >= Protocol.CURR_VERSION.length) + throw new IllegalArgumentException("Latest version for API key " + this + " is not defined"); + return Protocol.CURR_VERSION[id]; + } + + public short oldestVersion() { + if (id >= Protocol.MIN_VERSIONS.length) + throw new IllegalArgumentException("Oldest version for API key " + this + " is not defined"); + return Protocol.MIN_VERSIONS[id]; + } + + public Schema requestSchema(short version) { + return schemaFor(Protocol.REQUESTS, version); + } + + public Schema responseSchema(short version) { + return schemaFor(Protocol.RESPONSES, version); + } + + public Struct parseRequest(short version, ByteBuffer buffer) { + return requestSchema(version).read(buffer); + } + + public Struct parseResponse(short version, ByteBuffer buffer) { + return responseSchema(version).read(buffer); + } + + private Schema schemaFor(Schema[][] schemas, short version) { + if (id > schemas.length) + throw new IllegalArgumentException("No schema available for API key " + this); + if (version < 0 || version > latestVersion()) + throw new IllegalArgumentException("Invalid version for API key " + this + ": " + version); + + Schema[] versions = schemas[id]; + if (versions[version] == null) + throw new IllegalArgumentException("Unsupported version for API key " + this + ": " + version); + return versions[version]; + } + private static String toHtml() { final StringBuilder b = new StringBuilder(); b.append("\n"); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java deleted file mode 100644 index ab2ebb13d8aa6..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.protocol; - -import java.nio.ByteBuffer; - -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; - -public class ProtoUtils { - - private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) { - if (apiKey < 0 || apiKey > schemas.length) - throw new IllegalArgumentException("Invalid api key: " + apiKey); - Schema[] versions = schemas[apiKey]; - if (version < 0 || version > latestVersion(apiKey)) - throw new IllegalArgumentException("Invalid version for API key " + apiKey + ": " + version); - if (versions[version] == null) - throw new IllegalArgumentException("Unsupported version for API key " + apiKey + ": " + version); - return versions[version]; - } - - public static short latestVersion(int apiKey) { - if (apiKey < 0 || apiKey >= Protocol.CURR_VERSION.length) - throw new IllegalArgumentException("Invalid api key: " + apiKey); - return Protocol.CURR_VERSION[apiKey]; - } - - public static short oldestVersion(int apiKey) { - if (apiKey < 0 || apiKey >= Protocol.CURR_VERSION.length) - throw new IllegalArgumentException("Invalid api key: " + apiKey); - return Protocol.MIN_VERSIONS[apiKey]; - } - - public static Schema requestSchema(int apiKey, int version) { - return schemaFor(Protocol.REQUESTS, apiKey, version); - } - - public static Schema currentRequestSchema(int apiKey) { - return requestSchema(apiKey, latestVersion(apiKey)); - } - - public static Schema responseSchema(int apiKey, int version) { - return schemaFor(Protocol.RESPONSES, apiKey, version); - } - - public static Schema currentResponseSchema(int apiKey) { - return schemaFor(Protocol.RESPONSES, apiKey, latestVersion(apiKey)); - } - - public static Struct parseRequest(int apiKey, int version, ByteBuffer buffer) { - return requestSchema(apiKey, version).read(buffer); - } - - public static Struct parseResponse(int apiKey, int version, ByteBuffer buffer) { - return responseSchema(apiKey, version).read(buffer); - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index eea1916cb7710..e2403bc13cb9e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -44,7 +43,7 @@ public ApiKeys apiKey() { } public short desiredOrLatestVersion() { - return desiredVersion == null ? ProtoUtils.latestVersion(apiKey.id) : desiredVersion; + return desiredVersion == null ? apiKey.latestVersion() : desiredVersion; } public T build() { @@ -90,7 +89,7 @@ public ByteBuffer serialize(RequestHeader header) { */ public static RequestAndSize getRequest(int requestId, short version, ByteBuffer buffer) { ApiKeys apiKey = ApiKeys.forId(requestId); - Struct struct = ProtoUtils.parseRequest(apiKey.id, version, buffer); + Struct struct = apiKey.parseRequest(version, buffer); AbstractRequest request; switch (apiKey) { case PRODUCE: diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index a21e340080c7d..b018871843901 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -47,8 +47,7 @@ public ByteBuffer serialize(short version, ResponseHeader responseHeader) { protected abstract Struct toStruct(short version); - public static AbstractResponse getResponse(int requestId, Struct struct) { - ApiKeys apiKey = ApiKeys.forId(requestId); + public static AbstractResponse getResponse(ApiKeys apiKey, Struct struct) { switch (apiKey) { case PRODUCE: return new ProduceResponse(struct); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java index 7d409007a9cf3..fde07bfc404de 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -48,7 +47,7 @@ public ApiVersionsRequest(Struct struct, short version) { @Override protected Struct toStruct() { - return new Struct(ProtoUtils.requestSchema(ApiKeys.API_VERSIONS.id, version())); + return new Struct(ApiKeys.API_VERSIONS.requestSchema(version())); } @Override @@ -59,12 +58,12 @@ public AbstractResponse getErrorResponse(Throwable e) { return new ApiVersionsResponse(Errors.forException(e), Collections.emptyList()); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.API_VERSIONS.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.API_VERSIONS.latestVersion())); } } public static ApiVersionsRequest parse(ByteBuffer buffer, short version) { - return new ApiVersionsRequest(ProtoUtils.parseRequest(ApiKeys.API_VERSIONS.id, version, buffer), version); + return new ApiVersionsRequest(ApiKeys.API_VERSIONS.parseRequest(version, buffer), version); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 0066c084467e9..6fea9791ac5b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -48,6 +47,10 @@ public static final class ApiVersion { public final short minVersion; public final short maxVersion; + public ApiVersion(ApiKeys apiKey) { + this(apiKey.id, apiKey.oldestVersion(), apiKey.latestVersion()); + } + public ApiVersion(short apiKey, short minVersion, short maxVersion) { this.apiKey = apiKey; this.minVersion = minVersion; @@ -84,7 +87,7 @@ public ApiVersionsResponse(Struct struct) { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.API_VERSIONS.id, version)); + Struct struct = new Struct(ApiKeys.API_VERSIONS.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); List apiVersionList = new ArrayList<>(); for (ApiVersion apiVersion : apiKeyToApiVersion.values()) { @@ -120,13 +123,13 @@ public Errors error() { } public static ApiVersionsResponse parse(ByteBuffer buffer, short version) { - return new ApiVersionsResponse(ProtoUtils.responseSchema(ApiKeys.API_VERSIONS.id, version).read(buffer)); + return new ApiVersionsResponse(ApiKeys.API_VERSIONS.responseSchema(version).read(buffer)); } private static ApiVersionsResponse createApiVersionsResponse() { List versionList = new ArrayList<>(); for (ApiKeys apiKey : ApiKeys.values()) { - versionList.add(new ApiVersion(apiKey.id, ProtoUtils.oldestVersion(apiKey.id), ProtoUtils.latestVersion(apiKey.id))); + versionList.add(new ApiVersion(apiKey)); } return new ApiVersionsResponse(Errors.NONE, versionList); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java index 679e5dd327958..48f0a11b5da1d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -69,7 +68,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new ControlledShutdownResponse(Errors.forException(e), Collections.emptySet()); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.CONTROLLED_SHUTDOWN_KEY.latestVersion())); } } @@ -77,14 +76,14 @@ public int brokerId() { return brokerId; } - public static ControlledShutdownRequest parse(ByteBuffer buffer, short versionId) { + public static ControlledShutdownRequest parse(ByteBuffer buffer, short version) { return new ControlledShutdownRequest( - ProtoUtils.parseRequest(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, versionId, buffer), versionId); + ApiKeys.CONTROLLED_SHUTDOWN_KEY.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version())); + Struct struct = new Struct(ApiKeys.CONTROLLED_SHUTDOWN_KEY.requestSchema(version())); struct.set(BROKER_ID_KEY_NAME, brokerId); return struct; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java index a2cac6c9f096f..a407c5a6bf804 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -69,12 +68,12 @@ public Set partitionsRemaining() { } public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) { - return new ControlledShutdownResponse(ProtoUtils.parseResponse(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version, buffer)); + return new ControlledShutdownResponse(ApiKeys.CONTROLLED_SHUTDOWN_KEY.parseResponse(version, buffer)); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id, version)); + Struct struct = new Struct(ApiKeys.CONTROLLED_SHUTDOWN_KEY.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java index a8f8c5e9ece31..7e196878301ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -227,7 +226,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new CreateTopicsResponse(topicErrors); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CREATE_TOPICS.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.CREATE_TOPICS.latestVersion())); } } @@ -247,8 +246,8 @@ public Set duplicateTopics() { return this.duplicateTopics; } - public static CreateTopicsRequest parse(ByteBuffer buffer, short versionId) { - return new CreateTopicsRequest(ProtoUtils.parseRequest(ApiKeys.CREATE_TOPICS.id, versionId, buffer), versionId); + public static CreateTopicsRequest parse(ByteBuffer buffer, short version) { + return new CreateTopicsRequest(ApiKeys.CREATE_TOPICS.parseRequest(version, buffer), version); } /** @@ -257,7 +256,7 @@ public static CreateTopicsRequest parse(ByteBuffer buffer, short versionId) { @Override public Struct toStruct() { short version = version(); - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.CREATE_TOPICS.id, version)); + Struct struct = new Struct(ApiKeys.CREATE_TOPICS.requestSchema(version)); List createTopicRequestStructs = new ArrayList<>(topics.size()); for (Map.Entry entry : topics.entrySet()) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java index 01b7c2b17d577..f0986ffc29b61 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -106,7 +105,7 @@ public CreateTopicsResponse(Struct struct) { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version)); + Struct struct = new Struct(ApiKeys.CREATE_TOPICS.responseSchema(version)); List topicErrorsStructs = new ArrayList<>(errors.size()); for (Map.Entry topicError : errors.entrySet()) { @@ -127,6 +126,6 @@ public Map errors() { } public static CreateTopicsResponse parse(ByteBuffer buffer, short version) { - return new CreateTopicsResponse(ProtoUtils.responseSchema(ApiKeys.CREATE_TOPICS.id, version).read(buffer)); + return new CreateTopicsResponse(ApiKeys.CREATE_TOPICS.responseSchema(version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java index eea4aa93cd60d..1cb6eb887ac96 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -80,7 +79,7 @@ public DeleteTopicsRequest(Struct struct, short version) { @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.DELETE_TOPICS.id, version())); + Struct struct = new Struct(ApiKeys.DELETE_TOPICS.requestSchema(version())); struct.set(TOPICS_KEY_NAME, topics.toArray()); struct.set(TIMEOUT_KEY_NAME, timeout); return struct; @@ -97,7 +96,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new DeleteTopicsResponse(topicErrors); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - version(), this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id))); + version(), this.getClass().getSimpleName(), ApiKeys.DELETE_TOPICS.latestVersion())); } } @@ -109,8 +108,8 @@ public Integer timeout() { return this.timeout; } - public static DeleteTopicsRequest parse(ByteBuffer buffer, short versionId) { - return new DeleteTopicsRequest(ProtoUtils.parseRequest(ApiKeys.DELETE_TOPICS.id, versionId, buffer), versionId); + public static DeleteTopicsRequest parse(ByteBuffer buffer, short version) { + return new DeleteTopicsRequest(ApiKeys.DELETE_TOPICS.parseRequest(version, buffer), version); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java index c47d0982b248c..dc651e1a09f75 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -61,7 +60,7 @@ public DeleteTopicsResponse(Struct struct) { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.DELETE_TOPICS.id, version)); + Struct struct = new Struct(ApiKeys.DELETE_TOPICS.responseSchema(version)); List topicErrorCodeStructs = new ArrayList<>(errors.size()); for (Map.Entry topicError : errors.entrySet()) { Struct topicErrorCodeStruct = struct.instance(TOPIC_ERROR_CODES_KEY_NAME); @@ -78,6 +77,6 @@ public Map errors() { } public static DeleteTopicsResponse parse(ByteBuffer buffer, short version) { - return new DeleteTopicsResponse(ProtoUtils.responseSchema(ApiKeys.DELETE_TOPICS.id, version).read(buffer)); + return new DeleteTopicsResponse(ApiKeys.DELETE_TOPICS.responseSchema(version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java index 0f1337101dc81..d5bde068b89b1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -64,25 +63,25 @@ public List groupIds() { @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.DESCRIBE_GROUPS.id, version())); + Struct struct = new Struct(ApiKeys.DESCRIBE_GROUPS.requestSchema(version())); struct.set(GROUP_IDS_KEY_NAME, groupIds.toArray()); return struct; } @Override public AbstractResponse getErrorResponse(Throwable e) { - short versionId = version(); - switch (versionId) { + short version = version(); + switch (version) { case 0: return DescribeGroupsResponse.fromError(Errors.forException(e), groupIds); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.DESCRIBE_GROUPS.id))); + version, this.getClass().getSimpleName(), ApiKeys.DESCRIBE_GROUPS.latestVersion())); } } - public static DescribeGroupsRequest parse(ByteBuffer buffer, short versionId) { - return new DescribeGroupsRequest(ProtoUtils.parseRequest(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer), versionId); + public static DescribeGroupsRequest parse(ByteBuffer buffer, short version) { + return new DescribeGroupsRequest(ApiKeys.DESCRIBE_GROUPS.parseRequest(version, buffer), version); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java index 0dde987476de8..bab13ffabeccb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -190,7 +189,7 @@ public static DescribeGroupsResponse fromError(Errors error, List groupI @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.DESCRIBE_GROUPS.id, version)); + Struct struct = new Struct(ApiKeys.DESCRIBE_GROUPS.responseSchema(version)); List groupStructs = new ArrayList<>(); for (Map.Entry groupEntry : groups.entrySet()) { @@ -219,7 +218,7 @@ protected Struct toStruct(short version) { return struct; } - public static DescribeGroupsResponse parse(ByteBuffer buffer, short versionId) { - return new DescribeGroupsResponse(ProtoUtils.parseResponse(ApiKeys.DESCRIBE_GROUPS.id, versionId, buffer)); + public static DescribeGroupsResponse parse(ByteBuffer buffer, short version) { + return new DescribeGroupsResponse(ApiKeys.DESCRIBE_GROUPS.parseResponse(version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 55fd286d631f8..7418b8977e51e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.utils.Utils; @@ -219,14 +218,14 @@ public boolean isFromFollower() { return replicaId >= 0; } - public static FetchRequest parse(ByteBuffer buffer, short versionId) { - return new FetchRequest(ProtoUtils.parseRequest(ApiKeys.FETCH.id, versionId, buffer), versionId); + public static FetchRequest parse(ByteBuffer buffer, short version) { + return new FetchRequest(ApiKeys.FETCH.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { short version = version(); - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version)); + Struct struct = new Struct(ApiKeys.FETCH.requestSchema(version)); List> topicsData = TopicAndPartitionData.batchByTopic(fetchData); struct.set(REPLICA_ID_KEY_NAME, replicaId); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index dee6a5f53b496..25631a5ed921d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; @@ -158,7 +157,7 @@ public int throttleTimeMs() { } public static FetchResponse parse(ByteBuffer buffer, short version) { - return new FetchResponse(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version).read(buffer)); + return new FetchResponse(ApiKeys.FETCH.responseSchema(version).read(buffer)); } private static void addResponseData(Struct struct, int throttleTimeMs, String dest, List sends) { @@ -212,7 +211,7 @@ private static void addPartitionData(String dest, List sends, Struct parti } private static Struct toStruct(short version, LinkedHashMap responseData, int throttleTime) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version)); + Struct struct = new Struct(ApiKeys.FETCH.responseSchema(version)); List> topicsData = FetchRequest.TopicAndPartitionData.batchByTopic(responseData); List topicArray = new ArrayList<>(); for (FetchRequest.TopicAndPartitionData topicEntry: topicsData) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java index 83d6cba27a36d..dcd71567166a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -65,7 +64,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new GroupCoordinatorResponse(Errors.GROUP_COORDINATOR_NOT_AVAILABLE, Node.noNode()); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.GROUP_COORDINATOR.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.GROUP_COORDINATOR.latestVersion())); } } @@ -73,14 +72,13 @@ public String groupId() { return groupId; } - public static GroupCoordinatorRequest parse(ByteBuffer buffer, short versionId) { - return new GroupCoordinatorRequest(ProtoUtils.parseRequest(ApiKeys.GROUP_COORDINATOR.id, versionId, buffer), - versionId); + public static GroupCoordinatorRequest parse(ByteBuffer buffer, short version) { + return new GroupCoordinatorRequest(ApiKeys.GROUP_COORDINATOR.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.GROUP_COORDINATOR.id, version())); + Struct struct = new Struct(ApiKeys.GROUP_COORDINATOR.requestSchema(version())); struct.set(GROUP_ID_KEY_NAME, groupId); return struct; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java index c13cf3c3ab707..8a0f5cdfcbc3c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -66,7 +65,7 @@ public Node node() { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.GROUP_COORDINATOR.id, version)); + Struct struct = new Struct(ApiKeys.GROUP_COORDINATOR.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); coordinator.set(NODE_ID_KEY_NAME, node.id()); @@ -77,6 +76,6 @@ protected Struct toStruct(short version) { } public static GroupCoordinatorResponse parse(ByteBuffer buffer, short version) { - return new GroupCoordinatorResponse(ProtoUtils.parseResponse(ApiKeys.GROUP_COORDINATOR.id, version, buffer)); + return new GroupCoordinatorResponse(ApiKeys.GROUP_COORDINATOR.parseResponse(version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 444083075c51c..0b713de2c3fd8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -79,7 +78,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new HeartbeatResponse(Errors.forException(e)); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.HEARTBEAT.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.HEARTBEAT.latestVersion())); } } @@ -95,13 +94,13 @@ public String memberId() { return memberId; } - public static HeartbeatRequest parse(ByteBuffer buffer, short versionId) { - return new HeartbeatRequest(ProtoUtils.parseRequest(ApiKeys.HEARTBEAT.id, versionId, buffer), versionId); + public static HeartbeatRequest parse(ByteBuffer buffer, short version) { + return new HeartbeatRequest(ApiKeys.HEARTBEAT.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.HEARTBEAT.id, version())); + Struct struct = new Struct(ApiKeys.HEARTBEAT.requestSchema(version())); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); struct.set(MEMBER_ID_KEY_NAME, memberId); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index 4cca846b3f6bc..ac58a70237f4b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -49,12 +48,12 @@ public Errors error() { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.HEARTBEAT.id, version)); + Struct struct = new Struct(ApiKeys.HEARTBEAT.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); return struct; } public static HeartbeatResponse parse(ByteBuffer buffer, short version) { - return new HeartbeatResponse(ProtoUtils.parseResponse(ApiKeys.HEARTBEAT.id, version, buffer)); + return new HeartbeatResponse(ApiKeys.HEARTBEAT.parseResponse(version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 37906a626c74e..93f124ae07d34 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -161,7 +160,7 @@ public AbstractResponse getErrorResponse(Throwable e) { default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.JOIN_GROUP.latestVersion())); } } @@ -190,13 +189,13 @@ public String protocolType() { } public static JoinGroupRequest parse(ByteBuffer buffer, short version) { - return new JoinGroupRequest(ProtoUtils.parseRequest(ApiKeys.JOIN_GROUP.id, version, buffer), version); + return new JoinGroupRequest(ApiKeys.JOIN_GROUP.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { short version = version(); - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.JOIN_GROUP.id, version)); + Struct struct = new Struct(ApiKeys.JOIN_GROUP.requestSchema(version)); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); if (version >= 1) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index d2a323b157bf7..82f64456fba7a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -117,12 +116,12 @@ public Map members() { } public static JoinGroupResponse parse(ByteBuffer buffer, short version) { - return new JoinGroupResponse(ProtoUtils.parseResponse(ApiKeys.JOIN_GROUP.id, version, buffer)); + return new JoinGroupResponse(ApiKeys.JOIN_GROUP.parseResponse(version, buffer)); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.JOIN_GROUP.id, version)); + Struct struct = new Struct(ApiKeys.JOIN_GROUP.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); struct.set(GENERATION_ID_KEY_NAME, generationId); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index c564b437d9d76..942590e5844f1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -147,7 +146,7 @@ public LeaderAndIsrRequest(Struct struct, short version) { @Override protected Struct toStruct() { short version = version(); - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LEADER_AND_ISR.id, version)); + Struct struct = new Struct(ApiKeys.LEADER_AND_ISR.requestSchema(version)); struct.set(CONTROLLER_ID_KEY_NAME, controllerId); struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); @@ -193,7 +192,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new LeaderAndIsrResponse(Errors.NONE, responses); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LEADER_AND_ISR.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.LEADER_AND_ISR.latestVersion())); } } @@ -213,8 +212,8 @@ public Set liveLeaders() { return liveLeaders; } - public static LeaderAndIsrRequest parse(ByteBuffer buffer, short versionId) { - return new LeaderAndIsrRequest(ProtoUtils.parseRequest(ApiKeys.LEADER_AND_ISR.id, versionId, buffer), versionId); + public static LeaderAndIsrRequest parse(ByteBuffer buffer, short version) { + return new LeaderAndIsrRequest(ApiKeys.LEADER_AND_ISR.parseRequest(version, buffer), version); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java index 2b02daf2fd277..3ec1d0534fee3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -70,12 +69,12 @@ public Errors error() { } public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) { - return new LeaderAndIsrResponse(ProtoUtils.parseResponse(ApiKeys.LEADER_AND_ISR.id, version, buffer)); + return new LeaderAndIsrResponse(ApiKeys.LEADER_AND_ISR.parseResponse(version, buffer)); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LEADER_AND_ISR.id, version)); + Struct struct = new Struct(ApiKeys.LEADER_AND_ISR.responseSchema(version)); List responseDatas = new ArrayList<>(responses.size()); for (Map.Entry response : responses.entrySet()) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java index 16622e41e0336..f89675eaa01b4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java @@ -15,7 +15,6 @@ import java.nio.ByteBuffer; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; public class LeaveGroupRequest extends AbstractRequest { @@ -71,7 +70,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new LeaveGroupResponse(Errors.forException(e)); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LEAVE_GROUP.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.LEAVE_GROUP.latestVersion())); } } @@ -83,13 +82,13 @@ public String memberId() { return memberId; } - public static LeaveGroupRequest parse(ByteBuffer buffer, short versionId) { - return new LeaveGroupRequest(ProtoUtils.parseRequest(ApiKeys.LEAVE_GROUP.id, versionId, buffer), versionId); + public static LeaveGroupRequest parse(ByteBuffer buffer, short version) { + return new LeaveGroupRequest(ApiKeys.LEAVE_GROUP.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LEAVE_GROUP.id, version())); + Struct struct = new Struct(ApiKeys.LEAVE_GROUP.requestSchema(version())); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(MEMBER_ID_KEY_NAME, memberId); return struct; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java index a28816a5b0ecc..983027e58cdf6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -48,13 +47,13 @@ public Errors error() { @Override public Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LEAVE_GROUP.id, version)); + Struct struct = new Struct(ApiKeys.LEAVE_GROUP.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); return struct; } public static LeaveGroupResponse parse(ByteBuffer buffer, short versionId) { - return new LeaveGroupResponse(ProtoUtils.parseResponse(ApiKeys.LEAVE_GROUP.id, versionId, buffer)); + return new LeaveGroupResponse(ApiKeys.LEAVE_GROUP.parseResponse(versionId, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java index badb527f38a34..cca5399ce12d1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -53,16 +52,16 @@ public AbstractResponse getErrorResponse(Throwable e) { return new ListGroupsResponse(Errors.forException(e), Collections.emptyList()); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_GROUPS.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.LIST_GROUPS.latestVersion())); } } - public static ListGroupsRequest parse(ByteBuffer buffer, short versionId) { - return new ListGroupsRequest(ProtoUtils.parseRequest(ApiKeys.LIST_GROUPS.id, versionId, buffer), versionId); + public static ListGroupsRequest parse(ByteBuffer buffer, short version) { + return new ListGroupsRequest(ApiKeys.LIST_GROUPS.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - return new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_GROUPS.id, version())); + return new Struct(ApiKeys.LIST_GROUPS.requestSchema(version())); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index e05a4b1df5817..ae54d33dc3812 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -84,7 +83,7 @@ public String protocolType() { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_GROUPS.id, version)); + Struct struct = new Struct(ApiKeys.LIST_GROUPS.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); List groupList = new ArrayList<>(); for (Group group : groups) { @@ -101,8 +100,8 @@ public static ListGroupsResponse fromError(Errors error) { return new ListGroupsResponse(error, Collections.emptyList()); } - public static ListGroupsResponse parse(ByteBuffer buffer, short versionId) { - return new ListGroupsResponse(ProtoUtils.parseResponse(ApiKeys.LIST_GROUPS.id, versionId, buffer)); + public static ListGroupsResponse parse(ByteBuffer buffer, short version) { + return new ListGroupsResponse(ApiKeys.LIST_GROUPS.parseResponse(version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 3e2ad7c1571a6..aea16de9a18b6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.Utils; @@ -68,7 +67,9 @@ public static Builder forReplica(short desiredVersion, int replicaId) { return new Builder((short) 0, desiredVersion, replicaId); } - public static Builder forConsumer(short minVersion) { + public static Builder forConsumer(boolean requireTimestamp) { + // If we need a timestamp in the response, the minimum RPC version we can send is v1. Otherwise, v0 is OK. + short minVersion = requireTimestamp ? (short) 1 : (short) 0; return new Builder(minVersion, null, CONSUMER_REPLICA_ID); } @@ -226,7 +227,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new ListOffsetResponse(responseData); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.LIST_OFFSETS.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.LIST_OFFSETS.latestVersion())); } } @@ -247,14 +248,14 @@ public Set duplicatePartitions() { return duplicatePartitions; } - public static ListOffsetRequest parse(ByteBuffer buffer, short versionId) { - return new ListOffsetRequest(ProtoUtils.parseRequest(ApiKeys.LIST_OFFSETS.id, versionId, buffer), versionId); + public static ListOffsetRequest parse(ByteBuffer buffer, short version) { + return new ListOffsetRequest(ApiKeys.LIST_OFFSETS.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { short version = version(); - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.LIST_OFFSETS.id, version)); + Struct struct = new Struct(ApiKeys.LIST_OFFSETS.requestSchema(version)); Map targetTimes = partitionTimestamps == null ? offsetData : partitionTimestamps; Map> topicsData = CollectionUtils.groupDataByTopic(targetTimes); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index cb3bafcc4eb80..d4ab2d2982141 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.Utils; @@ -146,12 +145,12 @@ public Map responseData() { } public static ListOffsetResponse parse(ByteBuffer buffer, short version) { - return new ListOffsetResponse(ProtoUtils.parseResponse(ApiKeys.LIST_OFFSETS.id, version, buffer)); + return new ListOffsetResponse(ApiKeys.LIST_OFFSETS.parseResponse(version, buffer)); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.LIST_OFFSETS.id, version)); + Struct struct = new Struct(ApiKeys.LIST_OFFSETS.responseSchema(version)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index f31315fbc43cc..db85c34e1ad64 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -120,7 +119,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new MetadataResponse(Collections.emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.METADATA.latestVersion())); } } @@ -132,13 +131,13 @@ public List topics() { return topics; } - public static MetadataRequest parse(ByteBuffer buffer, short versionId) { - return new MetadataRequest(ProtoUtils.parseRequest(ApiKeys.METADATA.id, versionId, buffer), versionId); + public static MetadataRequest parse(ByteBuffer buffer, short version) { + return new MetadataRequest(ApiKeys.METADATA.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.METADATA.id, version())); + Struct struct = new Struct(ApiKeys.METADATA.requestSchema(version())); if (topics == null) struct.set(TOPICS_KEY_NAME, null); else diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 1532f67d571ed..fa79e92c3442d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -17,7 +17,6 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -257,7 +256,7 @@ public String clusterId() { } public static MetadataResponse parse(ByteBuffer buffer, short version) { - return new MetadataResponse(ProtoUtils.parseResponse(ApiKeys.METADATA.id, version, buffer)); + return new MetadataResponse(ApiKeys.METADATA.parseResponse(version, buffer)); } public static class TopicMetadata { @@ -337,7 +336,7 @@ public List isr() { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version)); + Struct struct = new Struct(ApiKeys.METADATA.responseSchema(version)); List brokerArray = new ArrayList<>(); for (Node node : brokers) { Struct broker = struct.instance(BROKERS_KEY_NAME); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index bf14f101a71b8..556292ca8fcc1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; @@ -209,7 +208,7 @@ public OffsetCommitRequest(Struct struct, short versionId) { @Override public Struct toStruct() { short version = version(); - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, version)); + Struct struct = new Struct(ApiKeys.OFFSET_COMMIT.requestSchema(version)); struct.set(GROUP_ID_KEY_NAME, groupId); Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); @@ -257,7 +256,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new OffsetCommitResponse(responseData); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_COMMIT.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.OFFSET_COMMIT.latestVersion())); } } @@ -281,8 +280,8 @@ public Map offsetData() { return offsetData; } - public static OffsetCommitRequest parse(ByteBuffer buffer, short versionId) { - Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId); - return new OffsetCommitRequest(schema.read(buffer), versionId); + public static OffsetCommitRequest parse(ByteBuffer buffer, short version) { + Schema schema = ApiKeys.OFFSET_COMMIT.requestSchema(version); + return new OffsetCommitRequest(schema.read(buffer), version); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index b5709e2dada45..8c0ff97575e2a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; @@ -75,7 +74,7 @@ public OffsetCommitResponse(Struct struct) { @Override public Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_COMMIT.id, version)); + Struct struct = new Struct(ApiKeys.OFFSET_COMMIT.responseSchema(version)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList<>(); @@ -102,7 +101,7 @@ public Map responseData() { } public static OffsetCommitResponse parse(ByteBuffer buffer, short version) { - return new OffsetCommitResponse(ProtoUtils.parseResponse(ApiKeys.OFFSET_COMMIT.id, version, buffer)); + return new OffsetCommitResponse(ApiKeys.OFFSET_COMMIT.parseResponse(version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 2a550e508236d..653cbd864e655 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.Utils; @@ -132,7 +131,7 @@ public OffsetFetchResponse getErrorResponse(Errors error) { return new OffsetFetchResponse(error, responsePartitions); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.OFFSET_FETCH.latestVersion())); } } @@ -149,8 +148,8 @@ public List partitions() { return partitions; } - public static OffsetFetchRequest parse(ByteBuffer buffer, short versionId) { - return new OffsetFetchRequest(ProtoUtils.parseRequest(ApiKeys.OFFSET_FETCH.id, versionId, buffer), versionId); + public static OffsetFetchRequest parse(ByteBuffer buffer, short version) { + return new OffsetFetchRequest(ApiKeys.OFFSET_FETCH.parseRequest(version, buffer), version); } public boolean isAllPartitions() { @@ -159,7 +158,7 @@ public boolean isAllPartitions() { @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_FETCH.id, version())); + Struct struct = new Struct(ApiKeys.OFFSET_FETCH.requestSchema(version())); struct.set(GROUP_ID_KEY_NAME, groupId); if (partitions != null) { Map> topicsData = CollectionUtils.groupDataByTopic(partitions); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 94de4b11a88c9..189575e2341a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; @@ -138,12 +137,12 @@ public Map responseData() { } public static OffsetFetchResponse parse(ByteBuffer buffer, short version) { - return new OffsetFetchResponse(ProtoUtils.parseResponse(ApiKeys.OFFSET_FETCH.id, version, buffer)); + return new OffsetFetchResponse(ApiKeys.OFFSET_FETCH.parseResponse(version, buffer)); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version)); + Struct struct = new Struct(ApiKeys.OFFSET_FETCH.responseSchema(version)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index df70e20bd6f42..7b454e867c3a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -17,7 +17,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.utils.CollectionUtils; @@ -107,7 +106,7 @@ public ProduceRequest(Struct struct, short version) { */ @Override public Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, version())); + Struct struct = new Struct(ApiKeys.PRODUCE.requestSchema(version())); Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); struct.set(ACKS_KEY_NAME, acks); struct.set(TIMEOUT_KEY_NAME, timeout); @@ -150,7 +149,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new ProduceResponse(responseMap); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.PRODUCE.latestVersion())); } } @@ -170,7 +169,7 @@ public void clearPartitionRecords() { partitionRecords.clear(); } - public static ProduceRequest parse(ByteBuffer buffer, short versionId) { - return new ProduceRequest(ProtoUtils.parseRequest(ApiKeys.PRODUCE.id, versionId, buffer), versionId); + public static ProduceRequest parse(ByteBuffer buffer, short version) { + return new ProduceRequest(ApiKeys.PRODUCE.parseRequest(version, buffer), version); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 7a022affc340a..823473226e329 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.utils.CollectionUtils; @@ -107,7 +106,7 @@ public ProduceResponse(Struct struct) { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version)); + Struct struct = new Struct(ApiKeys.PRODUCE.responseSchema(version)); Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList<>(responseByTopic.size()); @@ -174,6 +173,6 @@ public String toString() { } public static ProduceResponse parse(ByteBuffer buffer, short version) { - return new ProduceResponse(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version).read(buffer)); + return new ProduceResponse(ApiKeys.PRODUCE.responseSchema(version).read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java index a1f3f0efd35ad..9cc3f1fa008b6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; @@ -44,7 +43,7 @@ public class SaslHandshakeRequest extends AbstractRequest { private final String mechanism; public SaslHandshakeRequest(String mechanism) { - super(ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id)); + super(ApiKeys.SASL_HANDSHAKE.latestVersion()); this.mechanism = mechanism; } @@ -66,17 +65,17 @@ public AbstractResponse getErrorResponse(Throwable e) { return new SaslHandshakeResponse(Errors.forException(e), enabledMechanisms); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.SASL_HANDSHAKE.latestVersion())); } } - public static SaslHandshakeRequest parse(ByteBuffer buffer, short versionId) { - return new SaslHandshakeRequest(ProtoUtils.parseRequest(ApiKeys.SASL_HANDSHAKE.id, versionId, buffer), versionId); + public static SaslHandshakeRequest parse(ByteBuffer buffer, short version) { + return new SaslHandshakeRequest(ApiKeys.SASL_HANDSHAKE.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.SASL_HANDSHAKE.id, version())); + Struct struct = new Struct(ApiKeys.SASL_HANDSHAKE.requestSchema(version())); struct.set(MECHANISM_KEY_NAME, mechanism); return struct; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java index 9d38c6ab21a3f..e1a4c87caa3a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; @@ -66,7 +65,7 @@ public Errors error() { @Override public Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.SASL_HANDSHAKE.id, version)); + Struct struct = new Struct(ApiKeys.SASL_HANDSHAKE.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray()); return struct; @@ -77,7 +76,7 @@ public List enabledMechanisms() { } public static SaslHandshakeResponse parse(ByteBuffer buffer, short version) { - return new SaslHandshakeResponse(ProtoUtils.parseResponse(ApiKeys.SASL_HANDSHAKE.id, version, buffer)); + return new SaslHandshakeResponse(ApiKeys.SASL_HANDSHAKE.parseResponse(version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java index 91806f18f2b48..7b79cd8e5d25e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -113,7 +112,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new StopReplicaResponse(Errors.NONE, responses); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.STOP_REPLICA.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.STOP_REPLICA.latestVersion())); } } @@ -133,13 +132,13 @@ public Set partitions() { return partitions; } - public static StopReplicaRequest parse(ByteBuffer buffer, short versionId) { - return new StopReplicaRequest(ProtoUtils.parseRequest(ApiKeys.STOP_REPLICA.id, versionId, buffer), versionId); + public static StopReplicaRequest parse(ByteBuffer buffer, short version) { + return new StopReplicaRequest(ApiKeys.STOP_REPLICA.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.STOP_REPLICA.id, version())); + Struct struct = new Struct(ApiKeys.STOP_REPLICA.requestSchema(version())); struct.set(CONTROLLER_ID_KEY_NAME, controllerId); struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java index 5ae5cc1cab5be..617b1c6a46acc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -69,13 +68,13 @@ public Errors error() { return error; } - public static StopReplicaResponse parse(ByteBuffer buffer, short versionId) { - return new StopReplicaResponse(ProtoUtils.parseResponse(ApiKeys.STOP_REPLICA.id, versionId, buffer)); + public static StopReplicaResponse parse(ByteBuffer buffer, short version) { + return new StopReplicaResponse(ApiKeys.STOP_REPLICA.parseResponse(version, buffer)); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.STOP_REPLICA.id, version)); + Struct struct = new Struct(ApiKeys.STOP_REPLICA.responseSchema(version)); List responseDatas = new ArrayList<>(responses.size()); for (Map.Entry response : responses.entrySet()) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java index 7ad5c9a168d68..b55ccff17b435 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -108,7 +107,7 @@ public AbstractResponse getErrorResponse(Throwable e) { ByteBuffer.wrap(new byte[]{})); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.SYNC_GROUP.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.SYNC_GROUP.latestVersion())); } } @@ -128,13 +127,13 @@ public String memberId() { return memberId; } - public static SyncGroupRequest parse(ByteBuffer buffer, short versionId) { - return new SyncGroupRequest(ProtoUtils.parseRequest(ApiKeys.SYNC_GROUP.id, versionId, buffer), versionId); + public static SyncGroupRequest parse(ByteBuffer buffer, short version) { + return new SyncGroupRequest(ApiKeys.SYNC_GROUP.parseRequest(version, buffer), version); } @Override protected Struct toStruct() { - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.SYNC_GROUP.id, version())); + Struct struct = new Struct(ApiKeys.SYNC_GROUP.requestSchema(version())); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(GENERATION_ID_KEY_NAME, generationId); struct.set(MEMBER_ID_KEY_NAME, memberId); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java index ff198aa0063b3..148815c54f9da 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -62,14 +61,14 @@ public ByteBuffer memberAssignment() { @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.SYNC_GROUP.id, version)); + Struct struct = new Struct(ApiKeys.SYNC_GROUP.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); struct.set(MEMBER_ASSIGNMENT_KEY_NAME, memberState); return struct; } public static SyncGroupResponse parse(ByteBuffer buffer, short version) { - return new SyncGroupResponse(ProtoUtils.parseResponse(ApiKeys.SYNC_GROUP.id, version, buffer)); + return new SyncGroupResponse(ApiKeys.SYNC_GROUP.parseResponse(version, buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index 8dd852da6bee1..98bf83b3cfa9d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.Utils; @@ -226,7 +225,7 @@ public UpdateMetadataRequest(Struct struct, short versionId) { @Override protected Struct toStruct() { short version = version(); - Struct struct = new Struct(ProtoUtils.requestSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)); + Struct struct = new Struct(ApiKeys.UPDATE_METADATA_KEY.requestSchema(version)); struct.set(CONTROLLER_ID_KEY_NAME, controllerId); struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); @@ -288,7 +287,7 @@ public AbstractResponse getErrorResponse(Throwable e) { return new UpdateMetadataResponse(Errors.forException(e)); else throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id))); + versionId, this.getClass().getSimpleName(), ApiKeys.UPDATE_METADATA_KEY.latestVersion())); } public int controllerId() { @@ -307,9 +306,8 @@ public Set liveBrokers() { return liveBrokers; } - public static UpdateMetadataRequest parse(ByteBuffer buffer, short versionId) { - return new UpdateMetadataRequest(ProtoUtils.parseRequest(ApiKeys.UPDATE_METADATA_KEY.id, versionId, buffer), - versionId); + public static UpdateMetadataRequest parse(ByteBuffer buffer, short version) { + return new UpdateMetadataRequest(ApiKeys.UPDATE_METADATA_KEY.parseRequest(version, buffer), version); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java index 0032fca77a7d4..5b8b46d2f2dda 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java @@ -15,7 +15,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -44,12 +43,12 @@ public Errors error() { } public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) { - return new UpdateMetadataResponse(ProtoUtils.parseResponse(ApiKeys.UPDATE_METADATA_KEY.id, version, buffer)); + return new UpdateMetadataResponse(ApiKeys.UPDATE_METADATA_KEY.parseResponse(version, buffer)); } @Override protected Struct toStruct(short version) { - Struct struct = new Struct(ProtoUtils.responseSchema(ApiKeys.UPDATE_METADATA_KEY.id, version)); + Struct struct = new Struct(ApiKeys.UPDATE_METADATA_KEY.responseSchema(version)); struct.set(ERROR_CODE_KEY_NAME, error.code()); return struct; } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index c89cc24886bed..eb6a25914e3d6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.ApiVersionsResponse; @@ -134,7 +133,7 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { networkClient.poll(1, time.milliseconds()); assertEquals(1, networkClient.inFlightRequestCount()); ResponseHeader respHeader = new ResponseHeader(request.correlationId()); - Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); + Struct resp = new Struct(ApiKeys.PRODUCE.responseSchema(ApiKeys.PRODUCE.latestVersion())); resp.set("responses", new Object[0]); Struct responseHeaderStruct = respHeader.toStruct(); int size = responseHeaderStruct.sizeOf() + resp.sizeOf(); diff --git a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java index 53c47c833bfb7..9f0868f202850 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion; import org.junit.Assert; @@ -64,8 +63,7 @@ public void testVersionsToString() { } else if (apiKey == ApiKeys.DELETE_TOPICS) { versionList.add(new ApiVersion(apiKey.id, (short) 10000, (short) 10001)); } else { - versionList.add(new ApiVersion(apiKey.id, - ProtoUtils.oldestVersion(apiKey.id), ProtoUtils.latestVersion(apiKey.id))); + versionList.add(new ApiVersion(apiKey)); } } NodeApiVersions versions = new NodeApiVersions(versionList); @@ -80,15 +78,15 @@ public void testVersionsToString() { } else { bld.append(apiKey.name).append("("). append(apiKey.id).append("): "); - if (ProtoUtils.oldestVersion(apiKey.id) == - ProtoUtils.latestVersion(apiKey.id)) { - bld.append(ProtoUtils.oldestVersion(apiKey.id)); + if (apiKey.oldestVersion() == + apiKey.latestVersion()) { + bld.append(apiKey.oldestVersion()); } else { - bld.append(ProtoUtils.oldestVersion(apiKey.id)). + bld.append(apiKey.oldestVersion()). append(" to "). - append(ProtoUtils.latestVersion(apiKey.id)); + append(apiKey.latestVersion()); } - bld.append(" [usable: ").append(ProtoUtils.latestVersion(apiKey.id)). + bld.append(" [usable: ").append(apiKey.latestVersion()). append("]"); } prefix = ", "; @@ -129,7 +127,7 @@ public void testUsableVersionLatestVersions() { versionList.add(new ApiVersion((short) 100, (short) 0, (short) 1)); NodeApiVersions versions = new NodeApiVersions(versionList); for (ApiKeys apiKey: ApiKeys.values()) { - assertEquals(ProtoUtils.latestVersion(apiKey.id), versions.usableVersion(apiKey)); + assertEquals(apiKey.latestVersion(), versions.usableVersion(apiKey)); } } } diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index f177ae65e34f4..8917a226e0789 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -31,4 +31,9 @@ public void testForIdWithInvalidIdHigh() { ApiKeys.forId(10000); } + @Test(expected = IllegalArgumentException.class) + public void schemaVersionOutOfRange() { + ApiKeys.PRODUCE.requestSchema((short) Protocol.REQUESTS[ApiKeys.PRODUCE.id].length); + } + } diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java deleted file mode 100644 index 440ca491d9744..0000000000000 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.protocol; - -import org.junit.Test; - -public class ProtoUtilsTest { - @Test(expected = IllegalArgumentException.class) - public void schemaVersionOutOfRange() { - ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, Protocol.REQUESTS[ApiKeys.PRODUCE.id].length); - } -} diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index d0b9639806bd7..1367ba152d1f2 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.MemoryRecords; @@ -169,7 +168,7 @@ public void testResponseHeader() { } private void checkOlderFetchVersions() throws Exception { - int latestVersion = ProtoUtils.latestVersion(ApiKeys.FETCH.id); + int latestVersion = ApiKeys.FETCH.latestVersion(); for (int i = 0; i < latestVersion; ++i) { checkErrorResponse(createFetchRequest(i), new UnknownServerException()); checkRequest(createFetchRequest(i)); @@ -220,11 +219,11 @@ public void produceResponseVersionTest() { assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime()); assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime()); assertEquals("Throttle time must be 10", 10, v2Response.getThrottleTime()); - assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0), + assertEquals("Should use schema version 0", ApiKeys.PRODUCE.responseSchema((short) 0), v0Response.toStruct((short) 0).schema()); - assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 1), + assertEquals("Should use schema version 1", ApiKeys.PRODUCE.responseSchema((short) 1), v1Response.toStruct((short) 1).schema()); - assertEquals("Should use schema version 2", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 2), + assertEquals("Should use schema version 2", ApiKeys.PRODUCE.responseSchema((short) 2), v2Response.toStruct((short) 2).schema()); assertEquals("Response data does not match", responseData, v0Response.responses()); assertEquals("Response data does not match", responseData, v1Response.responses()); @@ -242,9 +241,9 @@ public void fetchResponseVersionTest() { FetchResponse v1Response = new FetchResponse(responseData, 10); assertEquals("Throttle time must be zero", 0, v0Response.throttleTimeMs()); assertEquals("Throttle time must be 10", 10, v1Response.throttleTimeMs()); - assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0), + assertEquals("Should use schema version 0", ApiKeys.FETCH.responseSchema((short) 0), v0Response.toStruct((short) 0).schema()); - assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.FETCH.id, 1), + assertEquals("Should use schema version 1", ApiKeys.FETCH.responseSchema((short) 1), v1Response.toStruct((short) 1).schema()); assertEquals("Response data does not match", responseData, v0Response.responseData()); assertEquals("Response data does not match", responseData, v1Response.responseData()); @@ -253,7 +252,7 @@ public void fetchResponseVersionTest() { @Test public void verifyFetchResponseFullWrite() throws Exception { FetchResponse fetchResponse = createFetchResponse(); - RequestHeader header = new RequestHeader(ApiKeys.FETCH.id, ProtoUtils.latestVersion(ApiKeys.FETCH.id), + RequestHeader header = new RequestHeader(ApiKeys.FETCH.id, ApiKeys.FETCH.latestVersion(), "client", 15); Send send = fetchResponse.toSend("1", header); @@ -272,7 +271,7 @@ public void verifyFetchResponseFullWrite() throws Exception { assertEquals(header.correlationId(), responseHeader.correlationId()); // read the body - Struct responseBody = ProtoUtils.responseSchema(ApiKeys.FETCH.id, header.apiVersion()).read(buf); + Struct responseBody = ApiKeys.FETCH.responseSchema(header.apiVersion()).read(buf); assertEquals(fetchResponse.toStruct(header.apiVersion()), responseBody); assertEquals(size, responseHeader.sizeOf() + responseBody.sizeOf()); @@ -281,7 +280,7 @@ public void verifyFetchResponseFullWrite() throws Exception { @Test public void testControlledShutdownResponse() { ControlledShutdownResponse response = createControlledShutdownResponse(); - short version = ProtoUtils.latestVersion(ApiKeys.CONTROLLED_SHUTDOWN_KEY.id); + short version = ApiKeys.CONTROLLED_SHUTDOWN_KEY.latestVersion(); Struct struct = response.toStruct(version); ByteBuffer buffer = toBuffer(struct); ControlledShutdownResponse deserialized = ControlledShutdownResponse.parse(buffer, version); @@ -403,11 +402,11 @@ private ListOffsetRequest createListOffsetRequest(int version) { Map offsetData = Collections.singletonMap( new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10)); - return ListOffsetRequest.Builder.forConsumer((short) 0).setOffsetData(offsetData).build((short) version); + return ListOffsetRequest.Builder.forConsumer(false).setOffsetData(offsetData).build((short) version); } else if (version == 1) { Map offsetData = Collections.singletonMap( new TopicPartition("test", 0), 1000000L); - return ListOffsetRequest.Builder.forConsumer((short) 1).setTargetTimes(offsetData).build((short) version); + return ListOffsetRequest.Builder.forConsumer(true).setTargetTimes(offsetData).build((short) version); } else { throw new IllegalArgumentException("Illegal ListOffsetRequest version " + version); } diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 3a9e0cec0d646..8a9353240ebc3 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; @@ -758,8 +757,8 @@ private void testUnauthenticatedApiVersionsRequest(SecurityProtocol securityProt // Send ApiVersionsRequest and check response ApiVersionsResponse versionsResponse = sendVersionRequestReceiveResponse(node); - assertEquals(ProtoUtils.oldestVersion(ApiKeys.SASL_HANDSHAKE.id), versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).minVersion); - assertEquals(ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id), versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).maxVersion); + assertEquals(ApiKeys.SASL_HANDSHAKE.oldestVersion(), versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).minVersion); + assertEquals(ApiKeys.SASL_HANDSHAKE.latestVersion(), versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).maxVersion); // Send SaslHandshakeRequest and check response SaslHandshakeResponse handshakeResponse = sendHandshakeRequestReceiveResponse(node); diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 4f71258219d3f..3368c090e2a44 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.errors._ -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.requests._ import CreateTopicsRequest.TopicDetails import org.apache.kafka.common.security.auth.KafkaPrincipal @@ -191,12 +191,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def createFetchRequest = { val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData] partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 100)) - val version = ProtoUtils.latestVersion(ApiKeys.FETCH.id) + val version = ApiKeys.FETCH.latestVersion requests.FetchRequest.Builder.forReplica(version, 5000, 100, Int.MaxValue, partitionMap).build() } private def createListOffsetsRequest = { - requests.ListOffsetRequest.Builder.forConsumer(0).setTargetTimes( + requests.ListOffsetRequest.Builder.forConsumer(false).setTargetTimes( Map(tp -> (0L: java.lang.Long)).asJava). build() } @@ -215,7 +215,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId, Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol, ListenerName.forSecurityProtocol(securityProtocol))).asJava, null)).asJava - val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion new requests.UpdateMetadataRequest.Builder(version, brokerId, Int.MaxValue, partitionState, brokers).build() } @@ -773,7 +773,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def testUnauthorizedDeleteWithoutDescribe() { val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val version = ApiKeys.DELETE_TOPICS.latestVersion val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, deleteResponse.errors.asScala.head._2) } @@ -782,7 +782,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { def testUnauthorizedDeleteWithDescribe() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), deleteTopicResource) val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val version = ApiKeys.DELETE_TOPICS.latestVersion val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, deleteResponse.errors.asScala.head._2) @@ -792,7 +792,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { def testDeleteWithWildCardAuth() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Delete)), new Resource(Topic, "*")) val response = connectAndSend(deleteTopicsRequest, ApiKeys.DELETE_TOPICS) - val version = ProtoUtils.latestVersion(ApiKeys.DELETE_TOPICS.id) + val version = ApiKeys.DELETE_TOPICS.latestVersion val deleteResponse = DeleteTopicsResponse.parse(response, version) assertEquals(Errors.NONE, deleteResponse.errors.asScala.head._2) diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index 448fce1a54c2e..b82ddf938d3f9 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -22,7 +22,7 @@ import java.util.Properties import kafka.network.SocketServer import kafka.utils.TestUtils import org.apache.kafka.common.protocol.types.Struct -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils} +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{CreateTopicsRequest, CreateTopicsResponse, MetadataRequest, MetadataResponse} import org.junit.Assert.{assertEquals, assertFalse, assertNotNull, assertTrue} @@ -147,9 +147,8 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { } protected def sendMetadataRequest(request: MetadataRequest, destination: SocketServer = anySocketServer): MetadataResponse = { - val version = ProtoUtils.latestVersion(ApiKeys.METADATA.id) val response = connectAndSend(request, ApiKeys.METADATA, destination = destination) - MetadataResponse.parse(response, version) + MetadataResponse.parse(response, ApiKeys.METADATA.latestVersion) } } diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala index 981ed6a69031b..2555a91b695de 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala @@ -18,7 +18,7 @@ package kafka.server import org.apache.kafka.common.requests.ApiVersionsResponse -import org.apache.kafka.common.protocol.{ApiKeys, ProtoUtils, Protocol} +import org.apache.kafka.common.protocol.{ApiKeys, Protocol} import org.junit.Assert._ import org.junit.Test @@ -32,8 +32,8 @@ class ApiVersionsTest { for (key <- ApiKeys.values) { val version = ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersion(key.id) assertNotNull(s"Could not find ApiVersion for API ${key.name}", version) - assertEquals(s"Incorrect min version for Api ${key.name}.", version.minVersion, ProtoUtils.oldestVersion(key.id)) - assertEquals(s"Incorrect max version for Api ${key.name}.", version.maxVersion, ProtoUtils.latestVersion(key.id)) + assertEquals(s"Incorrect min version for Api ${key.name}.", version.minVersion, key.oldestVersion) + assertEquals(s"Incorrect max version for Api ${key.name}.", version.maxVersion, key.latestVersion) // Check if versions less than min version are indeed set as null, i.e., deprecated. for (i <- 0 until version.minVersion) { diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 64be5b3088893..56169560ecb9b 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -24,7 +24,7 @@ import kafka.utils.TestUtils import kafka.utils.TestUtils._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils} +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.LogEntry import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer @@ -70,7 +70,7 @@ class FetchRequestTest extends BaseRequestTest { private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = { val response = connectAndSend(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) - FetchResponse.parse(response, ProtoUtils.latestVersion(ApiKeys.FETCH.id)) + FetchResponse.parse(response, ApiKeys.FETCH.latestVersion) } @Test diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 1b5007dbd41f7..57c1846a68bfa 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -22,7 +22,7 @@ import util.Arrays.asList import kafka.common.BrokerEndPointNotAvailableException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.requests.{PartitionState, UpdateMetadataRequest} import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint} import org.junit.Test @@ -69,7 +69,7 @@ class MetadataCacheTest { new TopicPartition(topic, 1) -> new PartitionState(controllerEpoch, 1, 1, asList(1), zkVersion, asSet(1)), new TopicPartition(topic, 2) -> new PartitionState(controllerEpoch, 2, 2, asList(2), zkVersion, asSet(2))) - val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) @@ -121,7 +121,7 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asSet(0))) - val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) @@ -165,7 +165,7 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) - val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) @@ -225,7 +225,7 @@ class MetadataCacheTest { val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) - val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) @@ -277,7 +277,7 @@ class MetadataCacheTest { val isr = asList[Integer](0, 1) val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) - val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) @@ -310,7 +310,7 @@ class MetadataCacheTest { val isr = asList[Integer](0, 1) val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) - val version = ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id) + val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() cache.updateCache(15, updateMetadataRequest) From 5a2abc51821fab4d3834b615e577d97e0f0e6494 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 23 Feb 2017 16:28:38 +0000 Subject: [PATCH 026/101] KAFKA-4788: Revert "KAFKA-4092: retention.bytes should not be allowed to be less than segment.bytes" The intent is good, but it needs to take into account broker configs as well. See KAFKA-4788 for more details. This reverts commit 4ca5abe8ee7578f602fb7653cb8a09640607ea85. Author: Ismael Juma Reviewers: Jun Rao Closes #2588 from ijuma/kafka-4788 --- core/src/main/scala/kafka/log/LogConfig.scala | 11 ----------- .../src/test/scala/unit/kafka/log/LogConfigTest.scala | 10 ---------- 2 files changed, 21 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index eaa60160c92e6..55669c061ec2c 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -323,23 +323,12 @@ object LogConfig { throw new InvalidConfigurationException(s"Unknown Log configuration $name.") } - /** - * Check that the property values are valid relative to each other - */ - def validateValues(props: Properties) { - val segmentBytes = if (props.getProperty(SegmentBytesProp) == null) Defaults.SegmentSize else props.getProperty(SegmentBytesProp).toLong - val retentionBytes = if (props.getProperty(RetentionBytesProp) == null) Defaults.RetentionSize else props.getProperty(RetentionBytesProp).toLong - if (segmentBytes > retentionBytes && retentionBytes != -1) - throw new InvalidConfigurationException(s"segment.bytes ${segmentBytes} is not less than or equal to retention.bytes ${retentionBytes}") - } - /** * Check that the given properties contain only valid log config names and that all values can be parsed and are valid */ def validate(props: Properties) { validateNames(props) configDef.parse(props) - validateValues(props) } } diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 5df76bce157c5..66702d683abcb 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -111,16 +111,6 @@ class LogConfigTest { case _: ConfigException => false } } - def testValueValidator() { - val p = new Properties() - p.setProperty(LogConfig.SegmentBytesProp, "100") - p.setProperty(LogConfig.RetentionBytesProp, "100") - LogConfig.validate(p) - p.setProperty(LogConfig.RetentionBytesProp, "90") - intercept[IllegalArgumentException] { - LogConfig.validate(p) - } - } private def assertPropertyInvalid(name: String, values: AnyRef*) { values.foreach((value) => { From 3428bc1a2549778378ffeb781ad62ca719ca3b7f Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Thu, 23 Feb 2017 10:04:34 -0800 Subject: [PATCH 027/101] MINOR: Reduce stream thread metrics overhead Reduces overheads by avoiding re-calling time.milliseconds(). Author: Eno Thereska Reviewers: Guozhang Wang Closes #2583 from enothereska/minor-reduce-milliseconds --- .../kafka/streams/processor/internals/StreamThread.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 12d472bc29783..e9a91f62554fe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -628,7 +628,7 @@ private void runLoop() { // only record poll latency is long poll is required if (longPoll) { - streamsMetrics.pollTimeSensor.record(computeLatency()); + streamsMetrics.pollTimeSensor.record(computeLatency(), timerStartedMs); } } @@ -644,7 +644,7 @@ private void runLoop() { requiresPoll = requiresPoll || task.requiresPoll(); - streamsMetrics.processTimeSensor.record(computeLatency()); + streamsMetrics.processTimeSensor.record(computeLatency(), timerStartedMs); maybePunctuate(task); @@ -721,7 +721,7 @@ private void maybePunctuate(StreamTask task) { // check whether we should punctuate based on the task's partition group timestamp; // which are essentially based on record timestamp. if (task.maybePunctuate()) - streamsMetrics.punctuateTimeSensor.record(computeLatency()); + streamsMetrics.punctuateTimeSensor.record(computeLatency(), timerStartedMs); } catch (KafkaException e) { log.error("{} Failed to punctuate active task {}: ", logPrefix, task.id(), e); @@ -786,7 +786,7 @@ private void commitOne(AbstractTask task) { throw e; } - streamsMetrics.commitTimeSensor.record(computeLatency()); + streamsMetrics.commitTimeSensor.record(computeLatency(), timerStartedMs); } /** From 6a55d41fe326d72052ca8214809c2e346f8ae775 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 23 Feb 2017 13:21:49 -0800 Subject: [PATCH 028/101] HOTFIX: add js template to upgrade.html Author: Guozhang Wang Reviewers: Ismael Juma Closes #2591 from guozhangwang/KHotfix-explicit-version --- docs/upgrade.html | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index ef9216ef195bf..2a6c1b76d9da7 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -15,6 +15,10 @@ limitations under the License. --> + + + + +
    \ No newline at end of file From 1ed4b48d5a43af743229fafc22f78846c9c3134d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 24 Feb 2017 11:44:31 +0000 Subject: [PATCH 029/101] KAFKA-4198; Fix race condition in KafkaServer.shutdown() It contained this step: val canShutdown = isShuttingDown.compareAndSet(false, true) if (canShutdown && shutdownLatch.getCount > 0) { without any fallback for the case of `shutdownLatch.getCount == 0`. So in the case of `shutdownLatch.getCount == 0` (when a previous call to the shutdown method was right about to finish) you would set `isShuttingDown` to true again without any possibility of ever getting the server started (since `startup` will check `isShuttingDown` before setting up a new latch with count 1). Long story short: concurrent calls to shutdown can get the server locked in a broken state. This fixes the reported error: java.lang.IllegalStateException: Kafka server is still shutting down, cannot re-start! at kafka.server.KafkaServer.startup(KafkaServer.scala:184) at kafka.integration.KafkaServerTestHarness$$anonfun$restartDeadBrokers$2.apply$mcVI$sp(KafkaServerTestHarness.scala:117) at kafka.integration.KafkaServerTestHarness$$anonfun$restartDeadBrokers$2.apply(KafkaServerTestHarness.scala:116) at kafka.integration.KafkaServerTestHarness$$anonfun$restartDeadBrokers$2.apply(KafkaServerTestHarness.scala:116) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733) at scala.collection.immutable.Range.foreach(Range.scala:160) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732) at kafka.integration.KafkaServerTestHarness$class.restartDeadBrokers(KafkaServerTestHarness.scala:116) at kafka.api.ConsumerBounceTest.restartDeadBrokers(ConsumerBounceTest.scala:34) at kafka.api.ConsumerBounceTest$BounceBrokerScheduler.doWork(ConsumerBounceTest.scala:158) Author: Armin Braun Reviewers: Ismael Juma Closes #2568 from original-brownbear/KAFKA-4198 --- .../org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 8 +++++--- .../scala/integration/kafka/api/ConsumerBounceTest.scala | 4 ++-- .../unit/kafka/integration/KafkaServerTestHarness.scala | 6 +++--- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 449efc9df180c..edb8dc18f8e5f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1495,7 +1495,7 @@ public Map endOffsets(Collection partition * If auto-commit is enabled, this will commit the current offsets if possible within the default * timeout. See {@link #close(long, TimeUnit)} for details. Note that {@link #wakeup()} * cannot be used to interrupt close. - * + * * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted * before or while this function is called */ diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index f5d8fc6a30448..fb8a48fd736ab 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -577,8 +577,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP if (isStartingUp.get) throw new IllegalStateException("Kafka server is still starting up, cannot shut down!") - val canShutdown = isShuttingDown.compareAndSet(false, true) - if (canShutdown && shutdownLatch.getCount > 0) { + // To ensure correct behavior under concurrent calls, we need to check `shutdownLatch` first since it gets updated + // last in the `if` block. If the order is reversed, we could shutdown twice or leave `isShuttingDown` set to + // `true` at the end of this method. + if (shutdownLatch.getCount > 0 && isShuttingDown.compareAndSet(false, true)) { CoreUtils.swallow(controlledShutdown()) brokerState.newState(BrokerShuttingDown) if(socketServer != null) @@ -608,7 +610,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP startupComplete.set(false) isShuttingDown.set(false) - AppInfoParser.unregisterAppInfo(jmxPrefix, config.brokerId.toString) + CoreUtils.swallow(AppInfoParser.unregisterAppInfo(jmxPrefix, config.brokerId.toString)) shutdownLatch.countDown() info("shut down completed") } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index fba3239b39407..2f37b5ba63ab4 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -13,8 +13,8 @@ package kafka.api +import java.util.concurrent._ import java.util.{Collection, Collections} -import java.util.concurrent.{Callable, Executors, ExecutorService, Future, Semaphore, TimeUnit} import kafka.admin.AdminClient import kafka.server.KafkaConfig @@ -23,7 +23,7 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.junit.Assert._ -import org.junit.{Before, After, Test} +import org.junit.{After, Before, Test} import scala.collection.JavaConverters._ diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index ff319143a944d..6fa7ad5147812 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -5,7 +5,7 @@ * 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 @@ -102,7 +102,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { } super.tearDown } - + /** * Pick a broker at random and kill it if it isn't already dead * Return the id of the broker killed @@ -120,7 +120,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { alive(index) = false } } - + /** * Restart any dead brokers */ From 01898ebbadec6d9df72bd9a89752d7f5bd128a7d Mon Sep 17 00:00:00 2001 From: Kamal C Date: Fri, 24 Feb 2017 13:42:33 +0000 Subject: [PATCH 030/101] MINOR: Code quality improvements to Config classes - Minor Javadoc fixes - Used final modifier if possible - Unnecessary type casts removed - Other minor clean-ups Author: Kamal C Reviewers: Colin P. Mccabe , Ismael Juma Closes #2578 from Kamal15/config --- .../kafka/common/config/AbstractConfig.java | 4 +- .../apache/kafka/common/config/Config.java | 2 +- .../apache/kafka/common/config/ConfigDef.java | 101 +++++++++--------- .../kafka/common/config/ConfigValue.java | 4 +- 4 files changed, 56 insertions(+), 55 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 6d985aea57757..bc852bd81c0b1 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -258,12 +258,12 @@ public List getConfiguredInstances(String key, Class t, Map) { o = Utils.newInstance((Class) klass); } else - throw new KafkaException("List contains element of type " + klass.getClass() + ", expected String or Class"); + throw new KafkaException("List contains element of type " + klass.getClass().getName() + ", expected String or Class"); if (!t.isInstance(o)) throw new KafkaException(klass + " is not an instance of " + t.getName()); if (o instanceof Configurable) diff --git a/clients/src/main/java/org/apache/kafka/common/config/Config.java b/clients/src/main/java/org/apache/kafka/common/config/Config.java index ce5ee17abdb0b..26f21d4f816b6 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/Config.java +++ b/clients/src/main/java/org/apache/kafka/common/config/Config.java @@ -15,7 +15,7 @@ import java.util.List; public class Config { - private List configValues; + private final List configValues; public Config(List configValues) { this.configValues = configValues; diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index e2e941a3323bd..25de8536d35fb 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -47,19 +47,20 @@ * * defs.define("config_with_default", Type.STRING, "default string value", "Configuration with default value."); * defs.define("config_with_validator", Type.INT, 42, Range.atLeast(0), "Configuration with user provided validator."); - * defs.define("config_with_dependents", Type.INT, "Configuration with dependents.", "group", 1, "Config With Dependents", Arrays.asList("config_with_default;","config_with_validator")); + * defs.define("config_with_dependents", Type.INT, "Configuration with dependents.", "group", 1, "Config With Dependents", Arrays.asList("config_with_default","config_with_validator")); * * Map<String, String> props = new HashMap<>(); * props.put("config_with_default", "some value"); * props.put("config_with_dependents", "some other value"); - * // will return "some value" + * * Map<String, Object> configs = defs.parse(props); + * // will return "some value" * String someConfig = (String) configs.get("config_with_default"); * // will return default value of 42 * int anotherConfig = (Integer) configs.get("config_with_validator"); * * To validate the full configuration, use: - * List<Config> configs = def.validate(props); + * List<Config> configs = defs.validate(props); * The {@link Config} contains updated configuration information given the current configuration values. * *

    @@ -497,10 +498,8 @@ private Map validate(Map parsed, Map undefinedDependentConfigs() { Set undefinedConfigKeys = new HashSet<>(); - for (String configName: configKeys.keySet()) { - ConfigKey configKey = configKeys.get(configName); - List dependents = configKey.dependents; - for (String dependent: dependents) { + for (ConfigKey configKey : configKeys.values()) { + for (String dependent: configKey.dependents) { if (!configKeys.containsKey(dependent)) { undefinedConfigKeys.add(dependent); } @@ -564,13 +563,14 @@ private void validate(String name, Map parsed, Map recommendedValues; + ConfigValue value = configs.get(name); + if (key.recommender != null) { try { - recommendedValues = key.recommender.validValues(name, parsed); - List originalRecommendedValues = config.recommendedValues(); + List recommendedValues = key.recommender.validValues(name, parsed); + List originalRecommendedValues = value.recommendedValues(); if (!originalRecommendedValues.isEmpty()) { Set originalRecommendedValueSet = new HashSet<>(originalRecommendedValues); Iterator it = recommendedValues.iterator(); @@ -581,14 +581,14 @@ private void validate(String name, Map parsed, Map) value; + return value; else if (value instanceof String) if (trimmed.isEmpty()) return Collections.emptyList(); @@ -678,7 +678,7 @@ else if (value instanceof String) throw new ConfigException(name, value, "Expected a comma separated list."); case CLASS: if (value instanceof Class) - return (Class) value; + return value; else if (value instanceof String) return Class.forName(trimmed, true, Utils.getContextOrKafkaClassLoader()); else @@ -777,6 +777,7 @@ public interface Validator { * Perform single configuration validation. * @param name The name of the configuration * @param value The value of the configuration + * @throws ConfigException if the value is invalid. */ void ensureValid(String name, Object value); } @@ -831,7 +832,7 @@ else if (max == null) public static class ValidList implements Validator { - ValidString validString; + final ValidString validString; private ValidList(List validStrings) { this.validString = new ValidString(validStrings); @@ -855,7 +856,7 @@ public String toString() { } public static class ValidString implements Validator { - List validStrings; + final List validStrings; private ValidString(List validStrings) { this.validStrings = validStrings; @@ -962,12 +963,12 @@ public String toHtmlTable() { b.append("\n"); } b.append("\n"); - for (ConfigKey def : configs) { + for (ConfigKey key : configs) { b.append("

    \n"); // print column values for (String headerName : headers()) { b.append(""); } b.append("\n"); @@ -982,8 +983,8 @@ public String toHtmlTable() { */ public String toRst() { StringBuilder b = new StringBuilder(); - for (ConfigKey def : sortedConfigs()) { - getConfigKeyRst(def, b); + for (ConfigKey key : sortedConfigs()) { + getConfigKeyRst(key, b); b.append("\n"); } return b.toString(); @@ -997,27 +998,27 @@ public String toEnrichedRst() { StringBuilder b = new StringBuilder(); String lastKeyGroupName = ""; - for (ConfigKey def : sortedConfigs()) { - if (def.group != null) { - if (!lastKeyGroupName.equalsIgnoreCase(def.group)) { - b.append(def.group).append("\n"); + for (ConfigKey key : sortedConfigs()) { + if (key.group != null) { + if (!lastKeyGroupName.equalsIgnoreCase(key.group)) { + b.append(key.group).append("\n"); - char[] underLine = new char[def.group.length()]; + char[] underLine = new char[key.group.length()]; Arrays.fill(underLine, '^'); b.append(new String(underLine)).append("\n\n"); } - lastKeyGroupName = def.group; + lastKeyGroupName = key.group; } - getConfigKeyRst(def, b); + getConfigKeyRst(key, b); - if (def.dependents != null && def.dependents.size() > 0) { + if (key.dependents != null && key.dependents.size() > 0) { int j = 0; b.append(" * Dependents: "); - for (String dependent : def.dependents) { + for (String dependent : key.dependents) { b.append("``"); b.append(dependent); - if (++j == def.dependents.size()) + if (++j == key.dependents.size()) b.append("``"); else b.append("``, "); @@ -1032,22 +1033,22 @@ public String toEnrichedRst() { /** * Shared content on Rst and Enriched Rst. */ - private void getConfigKeyRst(ConfigKey def, StringBuilder b) { - b.append("``").append(def.name).append("``").append("\n"); - for (String docLine : def.documentation.split("\n")) { + private void getConfigKeyRst(ConfigKey key, StringBuilder b) { + b.append("``").append(key.name).append("``").append("\n"); + for (String docLine : key.documentation.split("\n")) { if (docLine.length() == 0) { continue; } b.append(" ").append(docLine).append("\n\n"); } - b.append(" * Type: ").append(getConfigValue(def, "Type")).append("\n"); - if (def.hasDefault()) { - b.append(" * Default: ").append(getConfigValue(def, "Default")).append("\n"); + b.append(" * Type: ").append(getConfigValue(key, "Type")).append("\n"); + if (key.hasDefault()) { + b.append(" * Default: ").append(getConfigValue(key, "Default")).append("\n"); } - if (def.validator != null) { - b.append(" * Valid Values: ").append(getConfigValue(def, "Valid Values")).append("\n"); + if (key.validator != null) { + b.append(" * Valid Values: ").append(getConfigValue(key, "Valid Values")).append("\n"); } - b.append(" * Importance: ").append(getConfigValue(def, "Importance")).append("\n"); + b.append(" * Importance: ").append(getConfigValue(key, "Importance")).append("\n"); } /** @@ -1093,8 +1094,8 @@ public int compare(ConfigKey k1, ConfigKey k2) { public void embed(final String keyPrefix, final String groupPrefix, final int startingOrd, final ConfigDef child) { int orderInGroup = startingOrd; - for (ConfigDef.ConfigKey key : child.sortedConfigs()) { - define(new ConfigDef.ConfigKey( + for (ConfigKey key : child.sortedConfigs()) { + define(new ConfigKey( keyPrefix + key.name, key.type, key.defaultValue, @@ -1114,7 +1115,7 @@ public void embed(final String keyPrefix, final String groupPrefix, final int st /** * Returns a new validator instance that delegates to the base validator but unprefixes the config name along the way. */ - private static ConfigDef.Validator embeddedValidator(final String keyPrefix, final ConfigDef.Validator base) { + private static Validator embeddedValidator(final String keyPrefix, final Validator base) { if (base == null) return null; return new ConfigDef.Validator() { @Override @@ -1139,9 +1140,9 @@ private static List embeddedDependents(final String keyPrefix, final Lis /** * Returns a new recommender instance that delegates to the base recommender but unprefixes the input parameters along the way. */ - private static ConfigDef.Recommender embeddedRecommender(final String keyPrefix, final ConfigDef.Recommender base) { + private static Recommender embeddedRecommender(final String keyPrefix, final Recommender base) { if (base == null) return null; - return new ConfigDef.Recommender() { + return new Recommender() { private String unprefixed(String k) { return k.substring(keyPrefix.length()); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java index a62fcc2885372..fea51b719d83b 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java @@ -23,10 +23,10 @@ public class ConfigValue { - private String name; + private final String name; private Object value; private List recommendedValues; - private List errorMessages; + private final List errorMessages; private boolean visible; public ConfigValue(String name) { From 0be3a7226f816bd4b612477ea3b9916bddade290 Mon Sep 17 00:00:00 2001 From: Jean-Philippe Daigle Date: Fri, 24 Feb 2017 15:31:36 +0000 Subject: [PATCH 031/101] MINOR: fixed javadoc typo in KafkaProducer.partitionsFor Author: Jean-Philippe Daigle Reviewers: Ismael Juma Closes #2592 from jpdaigle/minor_typo_fix --- .../java/org/apache/kafka/clients/producer/KafkaProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 59916a7c7f53e..d8d94e6aaadf3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -625,7 +625,7 @@ public void flush() { } /** - * Get the partition metadata for the give topic. This can be used for custom partitioning. + * Get the partition metadata for the given topic. This can be used for custom partitioning. * @throws InterruptException If the thread is interrupted while blocked */ @Override From 2a7b18a2ac1213b189df8585671b713373dae383 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Fri, 24 Feb 2017 17:24:07 +0000 Subject: [PATCH 032/101] KAFKA-4779; Fix security upgrade system test to be non-disruptive The phase_two security upgrade test verifies upgrading inter-broker and client protocols to the same value as well as different values. The second case currently changes inter-broker protocol without first enabling the protocol, disrupting produce/consume until the whole cluster is updated. This commit changes the test to be a non-disruptive upgrade test that enables protocols first (simulating phase one of upgrade). Author: Rajini Sivaram Reviewers: Apurva Mehta , Ismael Juma Closes #2589 from rajinisivaram/KAFKA-4779 --- tests/kafkatest/services/kafka/kafka.py | 7 ++++++- tests/kafkatest/services/security/security_config.py | 4 ++++ .../kafkatest/tests/core/security_rolling_upgrade_test.py | 8 ++++---- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 2e65e099ac437..7e4a5899e6c48 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -126,9 +126,14 @@ def set_version(self, version): @property def security_config(self): - return SecurityConfig(self.context, self.security_protocol, self.interbroker_security_protocol, + config = SecurityConfig(self.context, self.security_protocol, self.interbroker_security_protocol, zk_sasl=self.zk.zk_sasl, client_sasl_mechanism=self.client_sasl_mechanism, interbroker_sasl_mechanism=self.interbroker_sasl_mechanism) + for protocol in self.port_mappings: + port = self.port_mappings[protocol] + if port.open: + config.enable_security_protocol(port.name) + return config def open_port(self, protocol): self.port_mappings[protocol] = self.port_mappings[protocol]._replace(open=True) diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py index 846d9b10f3a6a..beb37950ce139 100644 --- a/tests/kafkatest/services/security/security_config.py +++ b/tests/kafkatest/services/security/security_config.py @@ -166,6 +166,10 @@ def client_config(self, template_props="", node=None): static_jaas_conf = node is None or (self.has_sasl and self.has_ssl) return SecurityConfig(self.context, self.security_protocol, client_sasl_mechanism=self.client_sasl_mechanism, template_props=template_props, static_jaas_conf=static_jaas_conf) + def enable_security_protocol(self, security_protocol): + self.has_sasl = self.has_sasl or self.is_sasl(security_protocol) + self.has_ssl = self.has_ssl or self.is_ssl(security_protocol) + def setup_ssl(self, node): node.account.ssh("mkdir -p %s" % SecurityConfig.CONFIG_DIR, allow_fail=False) node.account.copy_to(SecurityConfig.ssl_stores.truststore_path, SecurityConfig.TRUSTSTORE_PATH) diff --git a/tests/kafkatest/tests/core/security_rolling_upgrade_test.py b/tests/kafkatest/tests/core/security_rolling_upgrade_test.py index a21e845649d71..a962a9d140222 100644 --- a/tests/kafkatest/tests/core/security_rolling_upgrade_test.py +++ b/tests/kafkatest/tests/core/security_rolling_upgrade_test.py @@ -69,8 +69,6 @@ def roll_in_secured_settings(self, client_protocol, broker_protocol): # Roll cluster to include inter broker security protocol. self.kafka.interbroker_security_protocol = broker_protocol - self.kafka.open_port(client_protocol) - self.kafka.open_port(broker_protocol) self.bounce() # Roll cluster to disable PLAINTEXT port @@ -131,14 +129,16 @@ def test_rolling_upgrade_phase_one(self, client_protocol): def test_rolling_upgrade_phase_two(self, client_protocol, broker_protocol): """ Start with a PLAINTEXT cluster with a second Secured port open (i.e. result of phase one). - Start an Producer and Consumer via the SECURED port - Incrementally upgrade to add inter-broker be the secure protocol + A third secure port is also open if inter-broker and client protocols are different. + Start a Producer and Consumer via the SECURED client port + Incrementally upgrade to add inter-broker be the secure broker protocol Incrementally upgrade again to add ACLs as well as disabling the PLAINTEXT port Ensure the producer and consumer ran throughout """ #Given we have a broker that has both secure and PLAINTEXT ports open self.kafka.security_protocol = client_protocol self.kafka.interbroker_security_protocol = "PLAINTEXT" + self.kafka.open_port(broker_protocol) self.kafka.start() #Create Secured Producer and Consumer From 151b23689ea1c2e13a1d38f403bf5839eddbe377 Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Fri, 24 Feb 2017 10:34:21 -0800 Subject: [PATCH 033/101] KAFKA-4494: Reduce startup and rebalance time by batching restoration of stores Replace one-by-one initialization of state stores with bulk initialization. Author: Damian Guy Reviewers: Eno Thereska, Guozhang Wang Closes #2560 from dguy/kafka-4494 --- .../kafka/clients/consumer/MockConsumer.java | 21 +- .../processor/internals/AbstractTask.java | 4 +- .../processor/internals/ChangelogReader.java | 52 ++ .../internals/GlobalStateManagerImpl.java | 1 - .../internals/ProcessorStateManager.java | 112 +---- .../processor/internals/StandbyTask.java | 5 +- .../processor/internals/StateRestorer.java | 78 +++ .../internals/StoreChangelogReader.java | 199 ++++++++ .../processor/internals/StreamTask.java | 4 +- .../processor/internals/StreamThread.java | 129 +++-- .../QueryableStateIntegrationTest.java | 2 +- .../processor/internals/AbstractTaskTest.java | 3 +- .../internals/ProcessorStateManagerTest.java | 473 ++---------------- .../processor/internals/StandbyTaskTest.java | 19 +- .../internals/StateRestorerTest.java | 70 +++ .../internals/StoreChangelogReaderTest.java | 258 ++++++++++ .../processor/internals/StreamTaskTest.java | 12 +- .../processor/internals/StreamThreadTest.java | 4 +- .../StreamThreadStateStoreProviderTest.java | 3 +- .../kafka/test/MockChangelogReader.java | 54 ++ .../kafka/test/MockRestoreCallback.java | 28 ++ .../kafka/test/MockRestoreConsumer.java | 155 ++++++ .../test/ProcessorTopologyTestDriver.java | 3 +- 23 files changed, 1067 insertions(+), 622 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java create mode 100644 streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java create mode 100644 streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java create mode 100644 streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index a88f4324ccb2d..9f312a71de5a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -152,17 +152,24 @@ public ConsumerRecords poll(long timeout) { updateFetchPosition(tp); // update the consumed offset + final Map>> results = new HashMap<>(); + for (final TopicPartition topicPartition : records.keySet()) { + results.put(topicPartition, new ArrayList>()); + } + for (Map.Entry>> entry : this.records.entrySet()) { if (!subscriptions.isPaused(entry.getKey())) { - List> recs = entry.getValue(); - if (!recs.isEmpty()) - this.subscriptions.position(entry.getKey(), recs.get(recs.size() - 1).offset() + 1); + final List> recs = entry.getValue(); + for (final ConsumerRecord rec : recs) { + if (assignment().contains(entry.getKey()) && rec.offset() >= subscriptions.position(entry.getKey())) { + results.get(entry.getKey()).add(rec); + subscriptions.position(entry.getKey(), rec.offset() + 1); + } + } } } - - ConsumerRecords copy = new ConsumerRecords(this.records); - this.records = new HashMap>>(); - return copy; + this.records.clear(); + return new ConsumerRecords<>(results); } public void addRecord(ConsumerRecord record) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index 8de5d2304d05a..be145d91919af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -57,7 +57,7 @@ protected AbstractTask(final TaskId id, final Collection partitions, final ProcessorTopology topology, final Consumer consumer, - final Consumer restoreConsumer, + final ChangelogReader changelogReader, final boolean isStandby, final StateDirectory stateDirectory, final ThreadCache cache) { @@ -70,7 +70,7 @@ protected AbstractTask(final TaskId id, // create the processor state manager try { - stateMgr = new ProcessorStateManager(id, partitions, restoreConsumer, isStandby, stateDirectory, topology.storeToChangelogTopic()); + stateMgr = new ProcessorStateManager(id, partitions, isStandby, stateDirectory, topology.storeToChangelogTopic(), changelogReader); } catch (IOException e) { throw new ProcessorStateException(String.format("task [%s] Error while creating the state manager", id), e); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java new file mode 100644 index 0000000000000..384eb6d466f6b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +/** + * Performs bulk read operations from a set of partitions. Used to + * restore {@link org.apache.kafka.streams.processor.StateStore}s from their + * change logs + */ +public interface ChangelogReader { + /** + * Validate that the partition exists on the cluster. + * @param topicPartition partition to validate. + * @param storeName name of the store the partition is for. + * @throws org.apache.kafka.streams.errors.StreamsException if partition doesn't exist + */ + void validatePartitionExists(final TopicPartition topicPartition, final String storeName); + + /** + * Register a state store and it's partition for later restoration. + * @param restorationInfo + */ + void register(final StateRestorer restorationInfo); + + /** + * Restore all registered state stores by reading from their changelogs. + */ + void restore(); + + /** + * @return the restored offsets for all persistent stores. + */ + Map restoredOffsets(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 3819bb5a09d70..1338387a767e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -131,7 +131,6 @@ public void register(final StateStore store, log.info("restoring state for global store {}", store.name()); final List topicPartitions = topicPartitionsForStore(store); - consumer.assign(topicPartitions); final Map highWatermarks = consumer.endOffsets(topicPartitions); try { restoreState(stateRestoreCallback, topicPartitions, highWatermarks); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 0e8caa2b76ed8..a06baed0c5b50 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -17,11 +17,8 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; @@ -36,13 +33,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import static java.util.Collections.singleton; public class ProcessorStateManager implements StateManager { @@ -56,9 +51,9 @@ public class ProcessorStateManager implements StateManager { private final String logPrefix; private final boolean isStandby; private final StateDirectory stateDirectory; + private final ChangelogReader changelogReader; private final Map stores; private final Map globalStores; - private final Consumer restoreConsumer; private final Map offsetLimits; private final Map restoredOffsets; private final Map checkpointedOffsets; @@ -77,12 +72,13 @@ public class ProcessorStateManager implements StateManager { */ public ProcessorStateManager(final TaskId taskId, final Collection sources, - final Consumer restoreConsumer, final boolean isStandby, final StateDirectory stateDirectory, - final Map storeToChangelogTopic) throws LockException, IOException { + final Map storeToChangelogTopic, + final ChangelogReader changelogReader) throws LockException, IOException { this.taskId = taskId; this.stateDirectory = stateDirectory; + this.changelogReader = changelogReader; this.baseDir = stateDirectory.directoryForTask(taskId); this.partitionForTopic = new HashMap<>(); for (TopicPartition source : sources) { @@ -90,7 +86,6 @@ public ProcessorStateManager(final TaskId taskId, } this.stores = new LinkedHashMap<>(); this.globalStores = new HashMap<>(); - this.restoreConsumer = restoreConsumer; this.offsetLimits = new HashMap<>(); this.restoredOffsets = new HashMap<>(); this.isStandby = isStandby; @@ -143,40 +138,8 @@ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallb return; } - // block until the partition is ready for this state changelog topic or time has elapsed - int partition = getPartition(topic); - boolean partitionNotFound = true; - long startTime = System.currentTimeMillis(); - long waitTime = 5000L; // hard-code the value since we should not block after KIP-4 - - do { - try { - Thread.sleep(50L); - } catch (InterruptedException e) { - // ignore - } - - List partitions; - try { - partitions = restoreConsumer.partitionsFor(topic); - } catch (TimeoutException e) { - throw new StreamsException(String.format("%s Could not fetch partition info for topic: %s before expiration of the configured request timeout", logPrefix, topic)); - } - if (partitions == null) { - throw new StreamsException(String.format("%s Could not find partition info for topic: %s", logPrefix, topic)); - } - for (PartitionInfo partitionInfo : partitions) { - if (partitionInfo.partition() == partition) { - partitionNotFound = false; - break; - } - } - } while (partitionNotFound && System.currentTimeMillis() < startTime + waitTime); - - if (partitionNotFound) { - throw new StreamsException(String.format("%s Store %s's change log (%s) does not contain partition %s", - logPrefix, store.name(), topic, partition)); - } + final TopicPartition storePartition = new TopicPartition(topic, getPartition(topic)); + changelogReader.validatePartitionExists(storePartition, store.name()); if (isStandby) { if (store.persistent()) { @@ -186,67 +149,17 @@ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallb } } else { log.trace("{} Restoring state store {} from changelog topic {}", logPrefix, store.name(), topic); - - restoreActiveState(topic, stateRestoreCallback); + final StateRestorer restorer = new StateRestorer(storePartition, + stateRestoreCallback, + checkpointedOffsets.get(storePartition), + offsetLimit(storePartition), + store.persistent()); + changelogReader.register(restorer); } this.stores.put(store.name(), store); } - private void restoreActiveState(String topicName, StateRestoreCallback stateRestoreCallback) { - // ---- try to restore the state from change-log ---- // - - // subscribe to the store's partition - if (!restoreConsumer.subscription().isEmpty()) { - throw new IllegalStateException(String.format("%s Restore consumer should have not subscribed to any partitions (%s) beforehand", logPrefix, restoreConsumer.subscription())); - } - TopicPartition storePartition = new TopicPartition(topicName, getPartition(topicName)); - restoreConsumer.assign(Collections.singletonList(storePartition)); - - try { - // calculate the end offset of the partition - // TODO: this is a bit hacky to first seek then position to get the end offset - restoreConsumer.seekToEnd(singleton(storePartition)); - long endOffset = restoreConsumer.position(storePartition); - - // restore from the checkpointed offset of the change log if it is persistent and the offset exists; - // restore the state from the beginning of the change log otherwise - if (checkpointedOffsets.containsKey(storePartition)) { - restoreConsumer.seek(storePartition, checkpointedOffsets.get(storePartition)); - } else { - restoreConsumer.seekToBeginning(singleton(storePartition)); - } - - // restore its state from changelog records - long limit = offsetLimit(storePartition); - while (true) { - long offset = 0L; - for (ConsumerRecord record : restoreConsumer.poll(100).records(storePartition)) { - offset = record.offset(); - if (offset >= limit) break; - stateRestoreCallback.restore(record.key(), record.value()); - } - - if (offset >= limit) { - break; - } else if (restoreConsumer.position(storePartition) == endOffset) { - break; - } else if (restoreConsumer.position(storePartition) > endOffset) { - // For a logging enabled changelog (no offset limit), - // the log end offset should not change while restoring since it is only written by this thread. - throw new IllegalStateException(String.format("%s Log end offset of %s should not change while restoring: old end offset %d, current offset %d", - logPrefix, storePartition, endOffset, restoreConsumer.position(storePartition))); - } - } - - // record the restored offset for its change log partition - long newOffset = Math.min(limit, restoreConsumer.position(storePartition)); - restoredOffsets.put(storePartition, newOffset); - } finally { - // un-assign the change log partition - restoreConsumer.assign(Collections.emptyList()); - } - } public Map checkpointed() { Map partitionsAndOffsets = new HashMap<>(); @@ -358,6 +271,7 @@ public void close(Map ackedOffsets) throws IOException { // write the checkpoint @Override public void checkpoint(final Map ackedOffsets) { + checkpointedOffsets.putAll(changelogReader.restoredOffsets()); for (String storeName : stores.keySet()) { // only checkpoint the offset to the offsets file if // it is persistent AND changelog enabled diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index a27098c34f22e..712740c59dfb7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -46,7 +46,6 @@ public class StandbyTask extends AbstractTask { * @param partitions the collection of assigned {@link TopicPartition} * @param topology the instance of {@link ProcessorTopology} * @param consumer the instance of {@link Consumer} - * @param restoreConsumer the instance of {@link Consumer} used when restoring state * @param config the {@link StreamsConfig} specified by the user * @param metrics the {@link StreamsMetrics} created by the thread * @param stateDirectory the {@link StateDirectory} created by the thread @@ -56,11 +55,11 @@ public StandbyTask(final TaskId id, final Collection partitions, final ProcessorTopology topology, final Consumer consumer, - final Consumer restoreConsumer, + final ChangelogReader changelogReader, final StreamsConfig config, final StreamsMetrics metrics, final StateDirectory stateDirectory) { - super(id, applicationId, partitions, topology, consumer, restoreConsumer, true, stateDirectory, null); + super(id, applicationId, partitions, topology, consumer, changelogReader, true, stateDirectory, null); // initialize the topology with its own context this.processorContext = new StandbyContextImpl(id, applicationId, config, stateMgr, metrics); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java new file mode 100644 index 0000000000000..b2de0f017597b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.processor.StateRestoreCallback; + +public class StateRestorer { + static final int NO_CHECKPOINT = -1; + private final TopicPartition partition; + private final StateRestoreCallback stateRestoreCallback; + private final Long checkpoint; + private final long offsetLimit; + private final boolean persistent; + private long restoredOffset; + + StateRestorer(final TopicPartition partition, + final StateRestoreCallback stateRestoreCallback, + final Long checkpoint, + final long offsetLimit, + final boolean persistent) { + this.partition = partition; + this.stateRestoreCallback = stateRestoreCallback; + this.checkpoint = checkpoint; + this.offsetLimit = offsetLimit; + this.persistent = persistent; + } + + public TopicPartition partition() { + return partition; + } + + public long checkpoint() { + return checkpoint == null ? NO_CHECKPOINT : checkpoint; + } + + public void restore(final byte[] key, final byte[] value) { + stateRestoreCallback.restore(key, value); + } + + public boolean isPersistent() { + return persistent; + } + + void setRestoredOffset(final long restoredOffset) { + this.restoredOffset = Math.min(offsetLimit, restoredOffset); + } + + boolean hasCompleted(final long recordOffset, final long endOffset) { + return endOffset == 0 || recordOffset >= readTo(endOffset); + } + + Long restoredOffset() { + return restoredOffset; + } + + long offsetLimit() { + return offsetLimit; + } + + private Long readTo(final long endOffset) { + return endOffset < offsetLimit ? endOffset : offsetLimit; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java new file mode 100644 index 0000000000000..ec9e4543d8e6b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -0,0 +1,199 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.StreamsException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class StoreChangelogReader implements ChangelogReader { + private static final Logger log = LoggerFactory.getLogger(StoreChangelogReader.class); + + private final Consumer consumer; + private final Time time; + private final long partitionValidationTimeoutMs; + private final Map> partitionInfo = new HashMap<>(); + private final Map stateRestorers = new HashMap<>(); + + + public StoreChangelogReader(final Consumer consumer, final Time time, final long partitionValidationTimeoutMs) { + this.consumer = consumer; + this.time = time; + this.partitionValidationTimeoutMs = partitionValidationTimeoutMs; + } + + + @Override + public void validatePartitionExists(final TopicPartition topicPartition, final String storeName) { + final long start = time.milliseconds(); + // fetch all info on all topics to avoid multiple remote calls + if (partitionInfo.isEmpty()) { + try { + partitionInfo.putAll(consumer.listTopics()); + } catch (final TimeoutException e) { + log.warn("Could not list topics so will fall back to partition by partition fetching"); + } + } + + final long endTime = time.milliseconds() + partitionValidationTimeoutMs; + while (!hasPartition(topicPartition) && time.milliseconds() < endTime) { + try { + final List partitions = consumer.partitionsFor(topicPartition.topic()); + if (partitions != null) { + partitionInfo.put(topicPartition.topic(), partitions); + } + } catch (final TimeoutException e) { + throw new StreamsException(String.format("Could not fetch partition info for topic: %s before expiration of the configured request timeout", + topicPartition.topic())); + } + } + + if (!hasPartition(topicPartition)) { + throw new StreamsException(String.format("Store %s's change log (%s) does not contain partition %s", + storeName, topicPartition.topic(), topicPartition.partition())); + } + log.debug("Took {} ms to validate that partition {} exists", time.milliseconds() - start, topicPartition); + } + + @Override + public void register(final StateRestorer restorer) { + if (restorer.offsetLimit() > 0) { + stateRestorers.put(restorer.partition(), restorer); + } + } + + public void restore() { + final long start = time.milliseconds(); + try { + if (!consumer.subscription().isEmpty()) { + throw new IllegalStateException(String.format("Restore consumer should have not subscribed to any partitions (%s) beforehand", consumer.subscription())); + } + final Map endOffsets = consumer.endOffsets(stateRestorers.keySet()); + + + // remove any partitions where we already have all of the data + final Map needsRestoring = new HashMap<>(); + for (final TopicPartition topicPartition : endOffsets.keySet()) { + final StateRestorer restorer = stateRestorers.get(topicPartition); + if (restorer.checkpoint() >= endOffsets.get(topicPartition)) { + restorer.setRestoredOffset(restorer.checkpoint()); + } else { + needsRestoring.put(topicPartition, restorer); + } + } + + consumer.assign(needsRestoring.keySet()); + + for (final StateRestorer restorer : needsRestoring.values()) { + if (restorer.checkpoint() != StateRestorer.NO_CHECKPOINT) { + consumer.seek(restorer.partition(), restorer.checkpoint()); + } else { + consumer.seekToBeginning(Collections.singletonList(restorer.partition())); + } + } + + final Set partitions = new HashSet<>(needsRestoring.keySet()); + while (!partitions.isEmpty()) { + final ConsumerRecords allRecords = consumer.poll(10); + final Iterator partitionIterator = partitions.iterator(); + while (partitionIterator.hasNext()) { + restorePartition(endOffsets, allRecords, partitionIterator); + } + } + } finally { + consumer.assign(Collections.emptyList()); + log.debug("Took {} ms to restore active state", time.milliseconds() - start); + } + } + + @Override + public Map restoredOffsets() { + final Map restoredOffsets = new HashMap<>(); + for (final Map.Entry entry : stateRestorers.entrySet()) { + final StateRestorer restorer = entry.getValue(); + if (restorer.isPersistent()) { + restoredOffsets.put(entry.getKey(), restorer.restoredOffset()); + } + } + return restoredOffsets; + } + + private void restorePartition(final Map endOffsets, + final ConsumerRecords allRecords, + final Iterator partitionIterator) { + final TopicPartition topicPartition = partitionIterator.next(); + final StateRestorer restorer = stateRestorers.get(topicPartition); + final Long endOffset = endOffsets.get(topicPartition); + final long pos = processNext(allRecords.records(topicPartition), restorer, endOffset); + if (restorer.hasCompleted(pos, endOffset)) { + if (pos > endOffset + 1) { + throw new IllegalStateException( + String.format("Log end offset of %s should not change while restoring: old end offset %d, current offset %d", + topicPartition, + endOffset, + pos)); + } + restorer.setRestoredOffset(pos); + partitionIterator.remove(); + } + } + + private long processNext(final List> records, final StateRestorer restorer, final Long endOffset) { + for (final ConsumerRecord record : records) { + final long offset = record.offset(); + if (restorer.hasCompleted(offset, endOffset)) { + return offset; + } + restorer.restore(record.key(), record.value()); + } + return consumer.position(restorer.partition()); + } + + private boolean hasPartition(final TopicPartition topicPartition) { + final List partitions = partitionInfo.get(topicPartition.topic()); + + if (partitions == null) { + return false; + } + + for (final PartitionInfo partition : partitions) { + if (partition.partition() == topicPartition.partition()) { + return true; + } + } + + return false; + + } + + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index d95ac4bac6fc1..983c07e3f8455 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -101,14 +101,14 @@ public StreamTask(TaskId id, Collection partitions, ProcessorTopology topology, Consumer consumer, - Consumer restoreConsumer, + final ChangelogReader changelogReader, StreamsConfig config, StreamsMetrics metrics, StateDirectory stateDirectory, ThreadCache cache, Time time, final RecordCollector recordCollector) { - super(id, applicationId, partitions, topology, consumer, restoreConsumer, false, stateDirectory, cache); + super(id, applicationId, partitions, topology, consumer, changelogReader, false, stateDirectory, cache); this.punctuationQueue = new PunctuationQueue(); this.maxBufferedSize = config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); this.metrics = new TaskMetrics(metrics); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index e9a91f62554fe..e5a4562c7d90f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -70,7 +70,6 @@ public class StreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(StreamThread.class); private static final AtomicInteger STREAM_THREAD_ID_SEQUENCE = new AtomicInteger(1); - /** * Stream thread states are the possible states that a stream thread can be in. * A thread must only be in one state at a time @@ -213,60 +212,11 @@ private synchronized void setStateWhenNotInPendingShutdown(final State newState) private boolean processStandbyRecords = false; private ThreadCache cache; + private StoreChangelogReader storeChangelogReader; private final TaskCreator taskCreator = new TaskCreator(); - final ConsumerRebalanceListener rebalanceListener = new ConsumerRebalanceListener() { - @Override - public void onPartitionsAssigned(Collection assignment) { - - try { - if (state == State.PENDING_SHUTDOWN) { - log.info("stream-thread [{}] New partitions [{}] assigned while shutting down.", - StreamThread.this.getName(), assignment); - } - log.info("stream-thread [{}] New partitions [{}] assigned at the end of consumer rebalance.", - StreamThread.this.getName(), assignment); - - setStateWhenNotInPendingShutdown(State.ASSIGNING_PARTITIONS); - // do this first as we may have suspended standby tasks that - // will become active or vice versa - closeNonAssignedSuspendedStandbyTasks(); - closeNonAssignedSuspendedTasks(); - addStreamTasks(assignment); - addStandbyTasks(); - lastCleanMs = time.milliseconds(); // start the cleaning cycle - streamsMetadataState.onChange(partitionAssignor.getPartitionsByHostState(), partitionAssignor.clusterMetadata()); - setStateWhenNotInPendingShutdown(State.RUNNING); - } catch (Throwable t) { - rebalanceException = t; - throw t; - } - } - - @Override - public void onPartitionsRevoked(Collection assignment) { - try { - if (state == State.PENDING_SHUTDOWN) { - log.info("stream-thread [{}] New partitions [{}] revoked while shutting down.", - StreamThread.this.getName(), assignment); - } - log.info("stream-thread [{}] partitions [{}] revoked at the beginning of consumer rebalance.", - StreamThread.this.getName(), assignment); - setStateWhenNotInPendingShutdown(State.PARTITIONS_REVOKED); - lastCleanMs = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned - // suspend active tasks - suspendTasksAndState(); - } catch (Throwable t) { - rebalanceException = t; - throw t; - } finally { - streamsMetadataState.onChange(Collections.>emptyMap(), partitionAssignor.clusterMetadata()); - removeStreamTasks(); - removeStandbyTasks(); - } - } - }; + final ConsumerRebalanceListener rebalanceListener; public synchronized boolean isInitialized() { return state == State.RUNNING; @@ -323,7 +273,6 @@ public StreamThread(TopologyBuilder builder, this.consumer = clientSupplier.getConsumer(consumerConfigs); log.info("{} Creating restore consumer client", logPrefix); this.restoreConsumer = clientSupplier.getRestoreConsumer(config.getRestoreConsumerConfigs(threadClientId)); - // initialize the task list // activeTasks needs to be concurrent as it can be accessed // by QueryableState @@ -347,7 +296,9 @@ public StreamThread(TopologyBuilder builder, this.timerStartedMs = time.milliseconds(); this.lastCleanMs = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment this.lastCommitMs = timerStartedMs; + this.rebalanceListener = new RebalanceListener(time, config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG)); setState(state.RUNNING); + } public void partitionAssignor(StreamPartitionAssignor partitionAssignor) { @@ -833,9 +784,15 @@ protected StreamTask createStreamTask(TaskId id, Collection part final ProcessorTopology topology = builder.build(id.topicGroupId); final RecordCollector recordCollector = new RecordCollectorImpl(producer, id.toString()); - return new StreamTask(id, applicationId, partitions, topology, consumer, restoreConsumer, config, streamsMetrics, stateDirectory, cache, time, recordCollector); + final long start = time.milliseconds(); + try { + return new StreamTask(id, applicationId, partitions, topology, consumer, storeChangelogReader, config, streamsMetrics, stateDirectory, cache, time, recordCollector); + } finally { + log.debug("{} creation of active task {} took {} ms", logPrefix, id, time.milliseconds() - start); + } } + private StreamTask findMatchingSuspendedTask(final TaskId taskId, final Set partitions) { if (suspendedTasks.containsKey(taskId)) { final StreamTask task = suspendedTasks.get(taskId); @@ -947,7 +904,7 @@ StandbyTask createStandbyTask(TaskId id, Collection partitions) ProcessorTopology topology = builder.build(id.topicGroupId); if (!topology.stateStores().isEmpty()) { - return new StandbyTask(id, applicationId, partitions, topology, consumer, restoreConsumer, config, streamsMetrics, stateDirectory); + return new StandbyTask(id, applicationId, partitions, topology, consumer, storeChangelogReader, config, streamsMetrics, stateDirectory); } else { return null; } @@ -1230,4 +1187,66 @@ void createTask(final TaskId taskId, final Set partitions) { } } + private class RebalanceListener implements ConsumerRebalanceListener { + private final Time time; + private final int requestTimeOut; + + RebalanceListener(final Time time, final int requestTimeOut) { + this.time = time; + this.requestTimeOut = requestTimeOut; + } + + @Override + public void onPartitionsAssigned(Collection assignment) { + final long start = time.milliseconds(); + try { + if (state == State.PENDING_SHUTDOWN) { + log.info("stream-thread [{}] New partitions [{}] assigned while shutting down.", + StreamThread.this.getName(), assignment); + } + log.info("stream-thread [{}] New partitions [{}] assigned at the end of consumer rebalance.", + StreamThread.this.getName(), assignment); + storeChangelogReader = new StoreChangelogReader(restoreConsumer, time, requestTimeOut); + setStateWhenNotInPendingShutdown(State.ASSIGNING_PARTITIONS); + // do this first as we may have suspended standby tasks that + // will become active or vice versa + closeNonAssignedSuspendedStandbyTasks(); + closeNonAssignedSuspendedTasks(); + addStreamTasks(assignment); + storeChangelogReader.restore(); + addStandbyTasks(); + streamsMetadataState.onChange(partitionAssignor.getPartitionsByHostState(), partitionAssignor.clusterMetadata()); + lastCleanMs = time.milliseconds(); // start the cleaning cycle + setStateWhenNotInPendingShutdown(State.RUNNING); + } catch (Throwable t) { + rebalanceException = t; + throw t; + } finally { + log.debug("{} partition assignment took {} ms", logPrefix, time.milliseconds() - start); + } + } + + @Override + public void onPartitionsRevoked(Collection assignment) { + try { + if (state == State.PENDING_SHUTDOWN) { + log.info("stream-thread [{}] New partitions [{}] revoked while shutting down.", + StreamThread.this.getName(), assignment); + } + log.info("stream-thread [{}] partitions [{}] revoked at the beginning of consumer rebalance.", + StreamThread.this.getName(), assignment); + setStateWhenNotInPendingShutdown(State.PARTITIONS_REVOKED); + lastCleanMs = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned + // suspend active tasks + suspendTasksAndState(); + } catch (Throwable t) { + rebalanceException = t; + throw t; + } finally { + streamsMetadataState.onChange(Collections.>emptyMap(), partitionAssignor.clusterMetadata()); + removeStreamTasks(); + removeStandbyTasks(); + } + } + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index 911c6a89cfe17..5e8b69e02bffd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -188,7 +188,7 @@ public int compare(final KeyValue o1, @After public void shutdown() throws IOException { if (kafkaStreams != null) { - kafkaStreams.close(); + kafkaStreams.close(30, TimeUnit.SECONDS); } IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java index f288f984d1ab1..dd2ed009fd4c3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; @@ -71,7 +72,7 @@ private AbstractTask createTask(final Consumer consumer) { Collections.emptyMap(), Collections.emptyList()), consumer, - consumer, + new StoreChangelogReader(consumer, Time.SYSTEM, 5000), false, new StateDirectory("app", TestUtils.tempDirectory().getPath(), time), new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index f1d3090b2c026..31a5d579eb386 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -18,25 +18,16 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.MockConsumer; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessorStateException; -import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; +import org.apache.kafka.test.MockChangelogReader; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.MockStateStoreSupplier; import org.apache.kafka.test.NoOpRecordCollector; @@ -50,14 +41,10 @@ import java.nio.channels.FileChannel; import java.nio.channels.FileLock; import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -65,139 +52,10 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; public class ProcessorStateManagerTest { - public static class MockRestoreConsumer extends MockConsumer { - private final Serializer serializer = new IntegerSerializer(); - - private TopicPartition assignedPartition = null; - private long seekOffset = -1L; - private boolean seekToBeginingCalled = false; - private boolean seekToEndCalled = false; - private long endOffset = 0L; - private long currentOffset = 0L; - - private ArrayList> recordBuffer = new ArrayList<>(); - - MockRestoreConsumer() { - super(OffsetResetStrategy.EARLIEST); - - reset(); - } - - // reset this mock restore consumer for a state store registration - public void reset() { - assignedPartition = null; - seekOffset = -1L; - seekToBeginingCalled = false; - seekToEndCalled = false; - endOffset = 0L; - recordBuffer.clear(); - } - - // buffer a record (we cannot use addRecord because we need to add records before assigning a partition) - public void bufferRecord(ConsumerRecord record) { - recordBuffer.add( - new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), 0L, - TimestampType.CREATE_TIME, 0L, 0, 0, - serializer.serialize(record.topic(), record.key()), - serializer.serialize(record.topic(), record.value()))); - endOffset = record.offset(); - - super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); - } - - @Override - public synchronized void assign(Collection partitions) { - int numPartitions = partitions.size(); - if (numPartitions > 1) - throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); - - if (numPartitions == 1) { - if (assignedPartition != null) - throw new IllegalStateException("RestoreConsumer: partition already assigned"); - assignedPartition = partitions.iterator().next(); - - // set the beginning offset to 0 - // NOTE: this is users responsible to set the initial lEO. - super.updateBeginningOffsets(Collections.singletonMap(assignedPartition, 0L)); - } - - super.assign(partitions); - } - - @Override - public ConsumerRecords poll(long timeout) { - // add buffered records to MockConsumer - for (ConsumerRecord record : recordBuffer) { - super.addRecord(record); - } - recordBuffer.clear(); - - ConsumerRecords records = super.poll(timeout); - - // set the current offset - Iterable> partitionRecords = records.records(assignedPartition); - for (ConsumerRecord record : partitionRecords) { - currentOffset = record.offset(); - } - - return records; - } - - @Override - public synchronized long position(TopicPartition partition) { - if (!partition.equals(assignedPartition)) - throw new IllegalStateException("RestoreConsumer: unassigned partition"); - - return currentOffset; - } - - @Override - public synchronized void seek(TopicPartition partition, long offset) { - if (offset < 0) - throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); - - if (seekOffset >= 0) - throw new IllegalStateException("RestoreConsumer: offset already seeked"); - - seekOffset = offset; - currentOffset = offset; - super.seek(partition, offset); - } - - @Override - public synchronized void seekToBeginning(Collection partitions) { - if (partitions.size() != 1) - throw new IllegalStateException("RestoreConsumer: other than one partition specified"); - - for (TopicPartition partition : partitions) { - if (!partition.equals(assignedPartition)) - throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); - } - - seekToBeginingCalled = true; - currentOffset = 0L; - } - - @Override - public synchronized void seekToEnd(Collection partitions) { - if (partitions.size() != 1) - throw new IllegalStateException("RestoreConsumer: other than one partition specified"); - - for (TopicPartition partition : partitions) { - if (!partition.equals(assignedPartition)) - throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); - } - - seekToEndCalled = true; - currentOffset = endOffset; - } - } - private final Set noPartitions = Collections.emptySet(); private final String applicationId = "test-application"; private final String persistentStoreName = "persistentStore"; @@ -211,7 +69,7 @@ public synchronized void seekToEnd(Collection partitions) { private final String changelogTopic = ProcessorStateManager.storeChangelogTopic(applicationId, storeName); private final TopicPartition changelogTopicPartition = new TopicPartition(changelogTopic, 0); private final TaskId taskId = new TaskId(0, 1); - private final MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + private final MockChangelogReader changelogReader = new MockChangelogReader(); private final MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(storeName, true); private File baseDir; private File checkpointFile; @@ -225,12 +83,6 @@ public void setup() { stateDirectory = new StateDirectory(applicationId, baseDir.getPath(), new MockTime()); checkpointFile = new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME); checkpoint = new OffsetCheckpoint(checkpointFile); - restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( - new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); - restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( - new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); } @After @@ -238,120 +90,39 @@ public void cleanup() { Utils.delete(baseDir); } - @Test(expected = StreamsException.class) - public void testNoTopic() throws IOException { - MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - - ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, new HashMap() { - { - put(nonPersistentStoreName, nonPersistentStoreName); - } - }); - - try { - stateMgr.register(mockStateStore, true, mockStateStore.stateRestoreCallback); - } finally { - stateMgr.close(Collections.emptyMap()); - } - } - @Test public void testRegisterPersistentStore() throws IOException { final TaskId taskId = new TaskId(0, 2); - long lastCheckpointedOffset = 10L; - - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME)); - checkpoint.write(Collections.singletonMap(new TopicPartition(persistentStoreTopicName, 2), lastCheckpointedOffset)); - - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); - - restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( - new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]), - new PartitionInfo(persistentStoreTopicName, 2, Node.noNode(), new Node[0], new Node[0]) - )); - - TopicPartition partition = new TopicPartition(persistentStoreTopicName, 2); - restoreConsumer.updateEndOffsets(Collections.singletonMap(partition, 13L)); MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore("persistentStore", true); // persistent store - - ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreName); } - }); + }, changelogReader); try { - restoreConsumer.reset(); - - ArrayList expectedKeys = new ArrayList<>(); - long offset; - for (int i = 1; i <= 3; i++) { - offset = (long) i; - int key = i * 10; - expectedKeys.add(key); - restoreConsumer.bufferRecord( - new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, TimestampType.CREATE_TIME, 0L, 0, 0, key, 0) - ); - } stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); - - assertEquals(new TopicPartition(persistentStoreTopicName, 2), restoreConsumer.assignedPartition); - assertEquals(lastCheckpointedOffset, restoreConsumer.seekOffset); - assertFalse(restoreConsumer.seekToBeginingCalled); - assertTrue(restoreConsumer.seekToEndCalled); - assertEquals(expectedKeys, persistentStore.keys); - + assertTrue(changelogReader.wasRegistered(new TopicPartition(persistentStoreTopicName, 2))); } finally { stateMgr.close(Collections.emptyMap()); } - - } @Test public void testRegisterNonPersistentStore() throws IOException { - long lastCheckpointedOffset = 10L; - - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); - checkpoint.write(Collections.singletonMap(new TopicPartition(persistentStoreTopicName, 2), lastCheckpointedOffset)); - - restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( - new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]), - new PartitionInfo(nonPersistentStoreTopicName, 2, Node.noNode(), new Node[0], new Node[0]) - )); - - TopicPartition partition = new TopicPartition(persistentStoreTopicName, 2); - restoreConsumer.updateEndOffsets(Collections.singletonMap(partition, 13L)); - - ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 2), noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); // non persistent store + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 2), noPartitions, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreTopicName); } - }); + }, changelogReader); try { - restoreConsumer.reset(); - - ArrayList expectedKeys = new ArrayList<>(); - long offset; - for (int i = 1; i <= 3; i++) { - offset = (long) (i + 100); - int key = i; - expectedKeys.add(i); - restoreConsumer.bufferRecord( - new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, TimestampType.CREATE_TIME, 0L, 0, 0, key, 0) - ); - } stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); - - assertEquals(new TopicPartition(nonPersistentStoreTopicName, 2), restoreConsumer.assignedPartition); - assertEquals(0L, restoreConsumer.seekOffset); - assertTrue(restoreConsumer.seekToBeginingCalled); - assertTrue(restoreConsumer.seekToEndCalled); - assertEquals(expectedKeys, nonPersistentStore.keys); + assertTrue(changelogReader.wasRegistered(new TopicPartition(nonPersistentStoreTopicName, 2))); } finally { stateMgr.close(Collections.emptyMap()); @@ -361,8 +132,6 @@ public void testRegisterNonPersistentStore() throws IOException { @Test public void testChangeLogOffsets() throws IOException { final TaskId taskId = new TaskId(0, 0); - final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint( - new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME)); long lastCheckpointedOffset = 10L; String storeName1 = "store1"; String storeName2 = "store2"; @@ -377,28 +146,13 @@ public void testChangeLogOffsets() throws IOException { storeToChangelogTopic.put(storeName2, storeTopicName2); storeToChangelogTopic.put(storeName3, storeTopicName3); - offsetCheckpoint.write(Collections.singletonMap(new TopicPartition(storeTopicName1, 0), lastCheckpointedOffset)); - - restoreConsumer.updatePartitions(storeTopicName1, Utils.mkList( - new PartitionInfo(storeTopicName1, 0, Node.noNode(), new Node[0], new Node[0]) - )); - restoreConsumer.updatePartitions(storeTopicName2, Utils.mkList( - new PartitionInfo(storeTopicName2, 0, Node.noNode(), new Node[0], new Node[0]) - )); - restoreConsumer.updatePartitions(storeTopicName3, Utils.mkList( - new PartitionInfo(storeTopicName3, 0, Node.noNode(), new Node[0], new Node[0]), - new PartitionInfo(storeTopicName3, 1, Node.noNode(), new Node[0], new Node[0]) - )); + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME)); + checkpoint.write(Collections.singletonMap(new TopicPartition(storeTopicName1, 0), lastCheckpointedOffset)); TopicPartition partition1 = new TopicPartition(storeTopicName1, 0); TopicPartition partition2 = new TopicPartition(storeTopicName2, 0); TopicPartition partition3 = new TopicPartition(storeTopicName3, 1); - Map endOffsets = new HashMap<>(); - endOffsets.put(partition1, 13L); - endOffsets.put(partition2, 17L); - restoreConsumer.updateEndOffsets(endOffsets); - MockStateStoreSupplier.MockStateStore store1 = new MockStateStoreSupplier.MockStateStore(storeName1, true); MockStateStoreSupplier.MockStateStore store2 = new MockStateStoreSupplier.MockStateStore(storeName2, true); MockStateStoreSupplier.MockStateStore store3 = new MockStateStoreSupplier.MockStateStore(storeName3, true); @@ -406,10 +160,8 @@ public void testChangeLogOffsets() throws IOException { // if there is an source partition, inherit the partition id Set sourcePartitions = Utils.mkSet(new TopicPartition(storeTopicName3, 1)); - ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, sourcePartitions, restoreConsumer, true, stateDirectory, storeToChangelogTopic); // standby + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, sourcePartitions, true, stateDirectory, storeToChangelogTopic, changelogReader); // standby try { - restoreConsumer.reset(); - stateMgr.register(store1, true, store1.stateRestoreCallback); stateMgr.register(store2, true, store2.stateRestoreCallback); stateMgr.register(store3, true, store3.stateRestoreCallback); @@ -427,17 +179,17 @@ public void testChangeLogOffsets() throws IOException { } finally { stateMgr.close(Collections.emptyMap()); } - } @Test public void testGetStore() throws IOException { - final ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + final MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); + final ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, false, stateDirectory, Collections.emptyMap(), changelogReader); try { - stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); + stateMgr.register(mockStateStore, true, mockStateStore.stateRestoreCallback); assertNull(stateMgr.getStore("noSuchStore")); - assertEquals(nonPersistentStore, stateMgr.getStore(nonPersistentStoreName)); + assertEquals(mockStateStore, stateMgr.getStore(nonPersistentStoreName)); } finally { stateMgr.close(Collections.emptyMap()); @@ -446,7 +198,6 @@ public void testGetStore() throws IOException { @Test public void testFlushAndClose() throws IOException { - // write an empty checkpoint file checkpoint.write(Collections.emptyMap()); // set up ack'ed offsets @@ -455,20 +206,17 @@ public void testFlushAndClose() throws IOException { ackedOffsets.put(new TopicPartition(nonPersistentStoreTopicName, 1), 456L); ackedOffsets.put(new TopicPartition(ProcessorStateManager.storeChangelogTopic(applicationId, "otherTopic"), 1), 789L); - ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreTopicName); } - }); + }, changelogReader); try { // make sure the checkpoint file isn't deleted assertTrue(checkpointFile.exists()); - restoreConsumer.reset(); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); - - restoreConsumer.reset(); stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); } finally { // close the state manager with the ack'ed offsets @@ -486,12 +234,12 @@ public void testFlushAndClose() throws IOException { // the checkpoint file should contain an offset from the persistent store only. final Map checkpointedOffsets = checkpoint.read(); assertEquals(1, checkpointedOffsets.size()); - assertEquals(new Long(123L + 1L), checkpointedOffsets.get(new TopicPartition(persistentStoreTopicName, 1))); + assertEquals(new Long(124), checkpointedOffsets.get(new TopicPartition(persistentStoreTopicName, 1))); } @Test public void shouldRegisterStoreWithoutLoggingEnabledAndNotBackedByATopic() throws Exception { - ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, Collections.emptyMap()); + final ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, false, stateDirectory, Collections.emptyMap(), changelogReader); stateMgr.register(nonPersistentStore, false, nonPersistentStore.stateRestoreCallback); assertNotNull(stateMgr.getStore(nonPersistentStoreName)); } @@ -501,14 +249,8 @@ public void shouldNotChangeOffsetsIfAckedOffsetsIsNull() throws Exception { final Map offsets = Collections.singletonMap(persistentStorePartition, 99L); checkpoint.write(offsets); - final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, - noPartitions, - restoreConsumer, - false, - stateDirectory, - Collections.emptyMap()); - - restoreConsumer.reset(); + final MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, false, stateDirectory, Collections.emptyMap(), changelogReader); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); stateMgr.close(null); final Map read = checkpoint.read(); @@ -519,12 +261,11 @@ public void shouldNotChangeOffsetsIfAckedOffsetsIsNull() throws Exception { public void shouldWriteCheckpointForPersistentLogEnabledStore() throws Exception { final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, - restoreConsumer, false, stateDirectory, Collections.singletonMap(persistentStore.name(), - persistentStoreTopicName)); - restoreConsumer.reset(); + persistentStoreTopicName), + changelogReader); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); @@ -537,22 +278,21 @@ public void shouldWriteCheckpointForPersistentLogEnabledStore() throws Exception public void shouldWriteCheckpointForStandbyReplica() throws Exception { final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, - restoreConsumer, true, stateDirectory, Collections.singletonMap(persistentStore.name(), - persistentStoreTopicName)); + persistentStoreTopicName), + changelogReader); - restoreConsumer.reset(); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); final byte[] bytes = Serdes.Integer().serializer().serialize("", 10); stateMgr.updateStandbyStates(persistentStorePartition, Collections.singletonList( new ConsumerRecord<>(persistentStorePartition.topic(), - persistentStorePartition.partition(), - 888L, - bytes, - bytes))); + persistentStorePartition.partition(), + 888L, + bytes, + bytes))); stateMgr.checkpoint(Collections.emptyMap()); @@ -565,19 +305,15 @@ public void shouldWriteCheckpointForStandbyReplica() throws Exception { public void shouldNotWriteCheckpointForNonPersistent() throws Exception { final TopicPartition topicPartition = new TopicPartition(nonPersistentStoreTopicName, 1); - restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( - new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, - restoreConsumer, true, stateDirectory, Collections.singletonMap(nonPersistentStoreName, - nonPersistentStoreTopicName)); + nonPersistentStoreTopicName), + changelogReader); - restoreConsumer.reset(); stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); stateMgr.checkpoint(Collections.singletonMap(topicPartition, 876L)); @@ -589,10 +325,10 @@ public void shouldNotWriteCheckpointForNonPersistent() throws Exception { public void shouldNotWriteCheckpointForStoresWithoutChangelogTopic() throws Exception { final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, - restoreConsumer, true, stateDirectory, - Collections.emptyMap()); + Collections.emptyMap(), + changelogReader); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); @@ -613,7 +349,7 @@ public void shouldThrowLockExceptionIfFailedToLockStateDirectory() throws Except final FileLock lock = channel.lock(); try { - new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + new ProcessorStateManager(taskId, noPartitions, false, stateDirectory, Collections.emptyMap(), changelogReader); fail("Should have thrown LockException"); } catch (final LockException e) { // pass @@ -627,10 +363,9 @@ public void shouldThrowLockExceptionIfFailedToLockStateDirectory() throws Except public void shouldThrowIllegalArgumentExceptionIfStoreNameIsSameAsCheckpointFileName() throws Exception { final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, noPartitions, - restoreConsumer, false, stateDirectory, - Collections.emptyMap()); + Collections.emptyMap(), changelogReader); try { stateManager.register(new MockStateStoreSupplier.MockStateStore(ProcessorStateManager.CHECKPOINT_FILE_NAME, true), true, null); @@ -644,10 +379,9 @@ public void shouldThrowIllegalArgumentExceptionIfStoreNameIsSameAsCheckpointFile public void shouldThrowIllegalArgumentExceptionOnRegisterWhenStoreHasAlreadyBeenRegistered() throws Exception { final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, noPartitions, - restoreConsumer, false, stateDirectory, - Collections.emptyMap()); + Collections.emptyMap(), changelogReader); stateManager.register(mockStateStore, false, null); try { @@ -659,142 +393,15 @@ public void shouldThrowIllegalArgumentExceptionOnRegisterWhenStoreHasAlreadyBeen } - @Test - public void shouldThrowStreamsExceptionWhenRestoreConsumerThrowsTimeoutException() throws Exception { - final MockRestoreConsumer mockRestoreConsumer = new MockRestoreConsumer() { - @Override - public List partitionsFor(final String topic) { - throw new TimeoutException("KABOOM!"); - } - }; - final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, - noPartitions, - mockRestoreConsumer, - false, - stateDirectory, - Collections.singletonMap(storeName, changelogTopic)); - try { - stateManager.register(mockStateStore, false, null); - fail("should have thrown StreamsException due to timeout exception"); - } catch (final StreamsException e) { - // pass - } - } - - @Test - public void shouldThrowStreamsExceptionWhenRestoreConsumerReturnsNullPartitions() throws Exception { - final MockRestoreConsumer mockRestoreConsumer = new MockRestoreConsumer() { - @Override - public List partitionsFor(final String topic) { - return null; - } - }; - final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, - noPartitions, - mockRestoreConsumer, - false, - stateDirectory, - Collections.singletonMap(storeName, changelogTopic)); - try { - stateManager.register(mockStateStore, false, null); - fail("should have thrown StreamsException due to timeout exception"); - } catch (final StreamsException e) { - // pass - } - } - - @Test - public void shouldThrowStreamsExceptionWhenPartitionForTopicNotFound() throws Exception { - final MockRestoreConsumer mockRestoreConsumer = new MockRestoreConsumer() { - @Override - public List partitionsFor(final String topic) { - return Collections.singletonList(new PartitionInfo(changelogTopic, 0, null, null, null)); - } - }; - final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, - Collections.singleton(new TopicPartition(changelogTopic, 1)), - mockRestoreConsumer, - false, - stateDirectory, - Collections.singletonMap(storeName, changelogTopic)); - - try { - stateManager.register(mockStateStore, false, null); - fail("should have thrown StreamsException due to partition for topic not found"); - } catch (final StreamsException e) { - // pass - } - } - - @Test - public void shouldThrowIllegalStateExceptionWhenRestoringStateAndSubscriptionsNonEmpty() throws Exception { - final MockRestoreConsumer mockRestoreConsumer = new MockRestoreConsumer() { - @Override - public List partitionsFor(final String topic) { - return Collections.singletonList(new PartitionInfo(changelogTopic, 0, null, null, null)); - } - }; - final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, - Collections.singleton(changelogTopicPartition), - mockRestoreConsumer, - false, - stateDirectory, - Collections.singletonMap(storeName, changelogTopic)); - - mockRestoreConsumer.subscribe(Collections.singleton("sometopic")); - - try { - stateManager.register(mockStateStore, false, null); - fail("should throw IllegalStateException when restore consumer has non-empty subscriptions"); - } catch (final IllegalStateException e) { - // pass - } - } - - @Test - public void shouldThrowIllegalStateExceptionWhenRestoreConsumerPositionGreaterThanEndOffset() throws Exception { - final AtomicInteger position = new AtomicInteger(10); - final MockRestoreConsumer mockRestoreConsumer = new MockRestoreConsumer() { - @Override - public synchronized long position(final TopicPartition partition) { - // need to make the end position change to trigger the exception - return position.getAndIncrement(); - } - }; - - mockRestoreConsumer.updatePartitions(changelogTopic, Collections.singletonList(new PartitionInfo(changelogTopic, 0, null, null, null))); - - final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, - Collections.singleton(changelogTopicPartition), - mockRestoreConsumer, - false, - stateDirectory, - Collections.singletonMap(storeName, changelogTopic)); - - stateManager.putOffsetLimit(changelogTopicPartition, 1); - // add a record with an offset less than the limit of 1 - mockRestoreConsumer.bufferRecord(new ConsumerRecord<>(changelogTopic, 0, 0, 1, 1)); - - - try { - stateManager.register(mockStateStore, false, mockStateStore.stateRestoreCallback); - fail("should have thrown IllegalStateException as end offset has changed"); - } catch (final IllegalStateException e) { - // pass - } - - } - @Test public void shouldThrowProcessorStateExceptionOnCloseIfStoreThrowsAnException() throws Exception { - restoreConsumer.updatePartitions(changelogTopic, Collections.singletonList(new PartitionInfo(changelogTopic, 0, null, null, null))); final ProcessorStateManager stateManager = new ProcessorStateManager(taskId, Collections.singleton(changelogTopicPartition), - restoreConsumer, false, stateDirectory, - Collections.singletonMap(storeName, changelogTopic)); + Collections.singletonMap(storeName, changelogTopic), + changelogReader); final MockStateStoreSupplier.MockStateStore stateStore = new MockStateStoreSupplier.MockStateStore(storeName, true) { @Override @@ -802,8 +409,6 @@ public void close() { throw new RuntimeException("KABOOM!"); } }; - stateManager.putOffsetLimit(changelogTopicPartition, 1); - restoreConsumer.bufferRecord(new ConsumerRecord<>(changelogTopic, 0, 1, 1, 1)); stateManager.register(stateStore, false, stateStore.stateRestoreCallback); try { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index ef4ebcc7c6a8d..31aa4a563e1b9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -30,12 +30,14 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; +import org.apache.kafka.test.MockRestoreConsumer; import org.apache.kafka.test.MockStateStoreSupplier; import org.apache.kafka.test.MockTimestampExtractor; import org.apache.kafka.test.TestUtils; @@ -123,7 +125,8 @@ private StreamsConfig createConfig(final File baseDir) throws Exception { } private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - private final ProcessorStateManagerTest.MockRestoreConsumer restoreStateConsumer = new ProcessorStateManagerTest.MockRestoreConsumer(); + private final MockRestoreConsumer restoreStateConsumer = new MockRestoreConsumer(); + private final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreStateConsumer, Time.SYSTEM, 5000); private final byte[] recordValue = intSerializer.serialize(null, 10); private final byte[] recordKey = intSerializer.serialize(null, 1); @@ -154,7 +157,7 @@ public void cleanup() { @Test public void testStorePartitions() throws Exception { StreamsConfig config = createConfig(baseDir); - StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, restoreStateConsumer, config, null, stateDirectory); + StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, changelogReader, config, null, stateDirectory); assertEquals(Utils.mkSet(partition2), new HashSet<>(task.changeLogPartitions())); @@ -164,7 +167,7 @@ public void testStorePartitions() throws Exception { @Test(expected = Exception.class) public void testUpdateNonPersistentStore() throws Exception { StreamsConfig config = createConfig(baseDir); - StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, restoreStateConsumer, config, null, stateDirectory); + StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, changelogReader, config, null, stateDirectory); restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); @@ -178,7 +181,7 @@ public void testUpdateNonPersistentStore() throws Exception { @Test public void testUpdate() throws Exception { StreamsConfig config = createConfig(baseDir); - StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, restoreStateConsumer, config, null, stateDirectory); + StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, changelogReader, config, null, stateDirectory); restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); @@ -236,7 +239,7 @@ public void testUpdateKTable() throws Exception { )); StreamsConfig config = createConfig(baseDir); - StandbyTask task = new StandbyTask(taskId, applicationId, ktablePartitions, ktableTopology, consumer, restoreStateConsumer, config, null, stateDirectory); + StandbyTask task = new StandbyTask(taskId, applicationId, ktablePartitions, ktableTopology, consumer, changelogReader, config, null, stateDirectory); restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); @@ -329,9 +332,9 @@ public void shouldNotThrowUnsupportedOperationExceptionWhenInitializingStateStor builder.stream("topic").groupByKey().count("my-store"); final ProcessorTopology topology = builder.setApplicationId(applicationId).build(0); StreamsConfig config = createConfig(baseDir); - new StandbyTask(taskId, applicationId, partitions, topology, consumer, restoreStateConsumer, config, - new MockStreamsMetrics(new Metrics()), stateDirectory); + new StandbyTask(taskId, applicationId, partitions, topology, consumer, changelogReader, config, + new MockStreamsMetrics(new Metrics()), stateDirectory); } @Test @@ -352,7 +355,7 @@ public void shouldCheckpointStoreOffsetsOnCommit() throws Exception { ktablePartitions, ktableTopology, consumer, - restoreStateConsumer, + changelogReader, config, null, stateDirectory diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java new file mode 100644 index 0000000000000..d457887c44ee3 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.test.MockRestoreCallback; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertTrue; + +public class StateRestorerTest { + + private static final long OFFSET_LIMIT = 50; + private final MockRestoreCallback callback = new MockRestoreCallback(); + private final StateRestorer restorer = new StateRestorer(new TopicPartition("topic", 1), callback, null, OFFSET_LIMIT, true); + + @Test + public void shouldCallRestoreOnRestoreCallback() throws Exception { + restorer.restore(new byte[0], new byte[0]); + assertThat(callback.restoreCount, equalTo(1)); + } + + @Test + public void shouldBeCompletedIfRecordOffsetGreaterThanEndOffset() throws Exception { + assertTrue(restorer.hasCompleted(11, 10)); + } + + @Test + public void shouldBeCompletedIfRecordOffsetGreaterThanOffsetLimit() throws Exception { + assertTrue(restorer.hasCompleted(51, 100)); + } + + @Test + public void shouldBeCompletedIfEndOffsetAndRecordOffsetAreZero() throws Exception { + assertTrue(restorer.hasCompleted(0, 0)); + } + + @Test + public void shouldBeCompletedIfOffsetAndOffsetLimitAreZero() throws Exception { + final StateRestorer restorer = new StateRestorer(new TopicPartition("topic", 1), callback, null, 0, true); + assertTrue(restorer.hasCompleted(0, 10)); + } + + @Test + public void shouldSetRestoredOffsetToMinOfLimitAndOffset() throws Exception { + restorer.setRestoredOffset(20); + assertThat(restorer.restoredOffset(), equalTo(20L)); + restorer.setRestoredOffset(100); + assertThat(restorer.restoredOffset(), equalTo(OFFSET_LIMIT)); + } + + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java new file mode 100644 index 0000000000000..d6fae66bd7018 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -0,0 +1,258 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.test.MockRestoreCallback; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.fail; + +public class StoreChangelogReaderTest { + + private final MockRestoreCallback callback = new MockRestoreCallback(); + private MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + private StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, new MockTime(), 0); + private final TopicPartition topicPartition = new TopicPartition("topic", 0); + private final PartitionInfo partitionInfo = new PartitionInfo(topicPartition.topic(), 0, null, null, null); + + @SuppressWarnings("unchecked") + @Test + public void shouldThrowStreamsExceptionWhenTimeoutExceptionThrown() throws Exception { + final MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST) { + @Override + public Map> listTopics() { + throw new TimeoutException("KABOOM!"); + } + }; + final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, new MockTime(), 0); + try { + changelogReader.validatePartitionExists(topicPartition, "store"); + fail("Should have thrown streams exception"); + } catch (final StreamsException e) { + // pass + } + } + + @Test(expected = StreamsException.class) + public void shouldThrowStreamsExceptionIfPartitionDoesntExistAfterMaxWait() throws Exception { + changelogReader.validatePartitionExists(topicPartition, "store"); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldFallbackToPartitionsForIfPartitionNotInAllPartitionsList() throws Exception { + final MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST) { + @Override + public List partitionsFor(final String topic) { + return Collections.singletonList(partitionInfo); + } + }; + + final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, new MockTime(), 10); + changelogReader.validatePartitionExists(topicPartition, "store"); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldThrowStreamsExceptionIfTimeoutOccursDuringPartitionsFor() throws Exception { + final MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST) { + @Override + public List partitionsFor(final String topic) { + throw new TimeoutException("KABOOM!"); + } + }; + final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, new MockTime(), 5); + try { + changelogReader.validatePartitionExists(topicPartition, "store"); + fail("Should have thrown streams exception"); + } catch (final StreamsException e) { + // pass + } + } + + @Test + public void shouldPassIfTopicPartitionExists() throws Exception { + consumer.updatePartitions(topicPartition.topic(), Collections.singletonList(partitionInfo)); + changelogReader.validatePartitionExists(topicPartition, "store"); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldRequestPartitionInfoIfItDoesntExist() throws Exception { + final MockConsumer consumer = new MockConsumer(OffsetResetStrategy.EARLIEST) { + @Override + public Map> listTopics() { + return Collections.emptyMap(); + } + }; + + consumer.updatePartitions(topicPartition.topic(), Collections.singletonList(partitionInfo)); + final StoreChangelogReader changelogReader = new StoreChangelogReader(consumer, Time.SYSTEM, 5000); + changelogReader.validatePartitionExists(topicPartition, "store"); + } + + + @Test + public void shouldThrowExceptionIfConsumerHasCurrentSubscription() throws Exception { + consumer.subscribe(Collections.singleton("sometopic")); + try { + changelogReader.restore(); + fail("Should have thrown IllegalStateException"); + } catch (final IllegalStateException e) { + // ok + } + } + + @Test + public void shouldRestoreAllMessagesFromBeginningWhenCheckpointNull() throws Exception { + final int messages = 10; + setupConsumer(messages, topicPartition); + changelogReader.register(new StateRestorer(topicPartition, callback, null, Long.MAX_VALUE, true)); + + changelogReader.restore(); + assertThat(callback.restoreCount, equalTo(messages)); + } + + @Test + public void shouldRestoreMessagesFromCheckpoint() throws Exception { + final int messages = 10; + setupConsumer(messages, topicPartition); + changelogReader.register(new StateRestorer(topicPartition, callback, 5L, Long.MAX_VALUE, true)); + + changelogReader.restore(); + assertThat(callback.restoreCount, equalTo(5)); + } + + @Test + public void shouldClearAssignmentAtEndOfRestore() throws Exception { + final int messages = 1; + setupConsumer(messages, topicPartition); + changelogReader.register(new StateRestorer(topicPartition, callback, null, Long.MAX_VALUE, true)); + + changelogReader.restore(); + assertThat(consumer.assignment(), equalTo(Collections.emptySet())); + } + + @Test + public void shouldRestoreToLimitWhenSupplied() throws Exception { + setupConsumer(10, topicPartition); + final StateRestorer restorer = new StateRestorer(topicPartition, callback, null, 3, true); + changelogReader.register(restorer); + + changelogReader.restore(); + assertThat(callback.restoreCount, equalTo(3)); + assertThat(restorer.restoredOffset(), equalTo(3L)); + } + + @Test + public void shouldRestoreMultipleStores() throws Exception { + final TopicPartition one = new TopicPartition("one", 0); + final TopicPartition two = new TopicPartition("two", 0); + final MockRestoreCallback callbackOne = new MockRestoreCallback(); + final MockRestoreCallback callbackTwo = new MockRestoreCallback(); + setupConsumer(10, topicPartition); + setupConsumer(5, one); + setupConsumer(3, two); + + changelogReader.register(new StateRestorer(topicPartition, callback, null, Long.MAX_VALUE, true)); + changelogReader.register(new StateRestorer(one, callbackOne, null, Long.MAX_VALUE, true)); + changelogReader.register(new StateRestorer(two, callbackTwo, null, Long.MAX_VALUE, true)); + + changelogReader.restore(); + + assertThat(callback.restoreCount, equalTo(10)); + assertThat(callbackOne.restoreCount, equalTo(5)); + assertThat(callbackTwo.restoreCount, equalTo(3)); + } + + @Test + public void shouldNotRestoreAnythingWhenPartitionIsEmpty() throws Exception { + final StateRestorer restorer = new StateRestorer(topicPartition, callback, null, Long.MAX_VALUE, true); + setupConsumer(0, topicPartition); + changelogReader.register(restorer); + + changelogReader.restore(); + assertThat(callback.restoreCount, equalTo(0)); + assertThat(restorer.restoredOffset(), equalTo(0L)); + } + + @Test + public void shouldNotRestoreAnythingWhenCheckpointAtEndOffset() throws Exception { + final Long endOffset = 10L; + setupConsumer(endOffset, topicPartition); + final StateRestorer restorer = new StateRestorer(topicPartition, callback, endOffset, Long.MAX_VALUE, true); + + changelogReader.register(restorer); + + changelogReader.restore(); + assertThat(callback.restoreCount, equalTo(0)); + assertThat(restorer.restoredOffset(), equalTo(endOffset)); + } + + @Test + public void shouldReturnRestoredOffsetsForPersistentStores() throws Exception { + setupConsumer(10, topicPartition); + changelogReader.register(new StateRestorer(topicPartition, callback, null, Long.MAX_VALUE, true)); + changelogReader.restore(); + final Map restoredOffsets = changelogReader.restoredOffsets(); + assertThat(restoredOffsets, equalTo(Collections.singletonMap(topicPartition, 10L))); + } + + @Test + public void shouldNotReturnRestoredOffsetsForNonPersistentStore() throws Exception { + setupConsumer(10, topicPartition); + changelogReader.register(new StateRestorer(topicPartition, callback, null, Long.MAX_VALUE, false)); + changelogReader.restore(); + final Map restoredOffsets = changelogReader.restoredOffsets(); + assertThat(restoredOffsets, equalTo(Collections.emptyMap())); + } + + private void setupConsumer(final long messages, final TopicPartition topicPartition) { + consumer.updatePartitions(topicPartition.topic(), + Collections.singletonList( + new PartitionInfo(topicPartition.topic(), + topicPartition.partition(), + null, + null, + null))); + consumer.updateBeginningOffsets(Collections.singletonMap(topicPartition, 0L)); + consumer.updateEndOffsets(Collections.singletonMap(topicPartition, Math.max(0, messages))); + consumer.assign(Collections.singletonList(topicPartition)); + + for (int i = 0; i < messages; i++) { + consumer.addRecord(new ConsumerRecord<>(topicPartition.topic(), topicPartition.partition(), i, new byte[0], new byte[0])); + } + consumer.assign(Collections.emptyList()); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 5c72fc9bff78c..14fda49d6ca88 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; @@ -103,6 +104,7 @@ public class StreamTaskTest { private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); private final MockConsumer restoreStateConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + private final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreStateConsumer, Time.SYSTEM, 5000); private final byte[] recordValue = intSerializer.serialize(null, 10); private final byte[] recordKey = intSerializer.serialize(null, 1); private final String applicationId = "applicationId"; @@ -141,7 +143,7 @@ public void setup() throws Exception { config = createConfig(baseDir); stateDirectory = new StateDirectory("applicationId", baseDir.getPath(), new MockTime()); task = new StreamTask(taskId00, applicationId, partitions, topology, consumer, - restoreStateConsumer, config, streamsMetrics, stateDirectory, null, time, recordCollector); + changelogReader, config, streamsMetrics, stateDirectory, null, time, recordCollector); } @After @@ -352,7 +354,7 @@ public void process(final Object key, final Object value) { task.close(); task = new StreamTask(taskId00, applicationId, partitions, - topology, consumer, restoreStateConsumer, config, streamsMetrics, stateDirectory, testCache, time, recordCollector); + topology, consumer, changelogReader, config, streamsMetrics, stateDirectory, testCache, time, recordCollector); final int offset = 20; task.addRecords(partition1, Collections.singletonList( new ConsumerRecord<>(partition1.topic(), partition1.partition(), offset, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue))); @@ -413,7 +415,7 @@ public void flush() { }; final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics()); final StreamTask streamTask = new StreamTask(taskId00, "appId", partitions, topology, consumer, - restoreStateConsumer, createConfig(baseDir), streamsMetrics, + changelogReader, createConfig(baseDir), streamsMetrics, stateDirectory, testCache, time, recordCollector); streamTask.flushState(); assertTrue(flushed.get()); @@ -462,7 +464,7 @@ public Map offsets() { final MockTime time = new MockTime(); final StreamsConfig config = createConfig(baseDir); final StreamTask streamTask = new StreamTask(taskId, "appId", partitions, topology, consumer, - restoreStateConsumer, config, streamsMetrics, + changelogReader, config, streamsMetrics, stateDirectory, new ThreadCache("testCache", 0, streamsMetrics), time, recordCollector); @@ -558,7 +560,7 @@ public void close() { return new StreamTask(taskId00, applicationId, partitions, - topology, consumer, restoreStateConsumer, config, streamsMetrics, stateDirectory, testCache, time, recordCollector); + topology, consumer, changelogReader, config, streamsMetrics, stateDirectory, testCache, time, recordCollector); } private Iterable> records(ConsumerRecord... recs) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index ef8dc92110887..38797100f4098 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -159,8 +159,8 @@ public TestStreamTask(TaskId id, StreamsConfig config, StreamsMetrics metrics, StateDirectory stateDirectory) { - super(id, applicationId, partitions, topology, consumer, restoreConsumer, config, metrics, - stateDirectory, null, new MockTime(), new RecordCollectorImpl(producer, id.toString())); + super(id, applicationId, partitions, topology, consumer, new StoreChangelogReader(restoreConsumer, Time.SYSTEM, 5000), config, metrics, + stateDirectory, null, new MockTime(), new RecordCollectorImpl(producer, id.toString())); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index dfe8d8b3a3d01..007cf7fd24d30 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.streams.processor.TopologyBuilder; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.StateDirectory; +import org.apache.kafka.streams.processor.internals.StoreChangelogReader; import org.apache.kafka.streams.processor.internals.StreamTask; import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.StreamsMetadataState; @@ -187,7 +188,7 @@ private StreamTask createStreamsTask(final String applicationId, return new StreamTask(taskId, applicationId, Collections .singletonList(new TopicPartition("topic", taskId.partition)), topology, clientSupplier.consumer, - clientSupplier.restoreConsumer, + new StoreChangelogReader(clientSupplier.restoreConsumer, Time.SYSTEM, 5000), streamsConfig, new MockStreamsMetrics(new Metrics()), stateDirectory, null, new MockTime(), new NoOpRecordCollector()) { @Override protected void initializeOffsetLimits() { diff --git a/streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java b/streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java new file mode 100644 index 0000000000000..afa87a896ae4d --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.test; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.processor.internals.ChangelogReader; +import org.apache.kafka.streams.processor.internals.StateRestorer; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public class MockChangelogReader implements ChangelogReader { + private final Set registered = new HashSet<>(); + + @Override + public void validatePartitionExists(final TopicPartition topicPartition, final String storeName) { + + } + + @Override + public void register(final StateRestorer restorationInfo) { + registered.add(restorationInfo.partition()); + } + + @Override + public void restore() { + + } + + @Override + public Map restoredOffsets() { + return Collections.emptyMap(); + } + + public boolean wasRegistered(final TopicPartition partition) { + return registered.contains(partition); + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java new file mode 100644 index 0000000000000..f61048eb5d43b --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.test; + +import org.apache.kafka.streams.processor.StateRestoreCallback; + +public class MockRestoreCallback implements StateRestoreCallback { + public int restoreCount = 0; + + @Override + public void restore(final byte[] key, final byte[] value) { + restoreCount++; + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java new file mode 100644 index 0000000000000..2178c16949143 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java @@ -0,0 +1,155 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.test; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; + +public class MockRestoreConsumer extends MockConsumer { + private final Serializer serializer = new IntegerSerializer(); + + private TopicPartition assignedPartition = null; + private long seekOffset = -1L; + private long endOffset = 0L; + private long currentOffset = 0L; + + private ArrayList> recordBuffer = new ArrayList<>(); + + public MockRestoreConsumer() { + super(OffsetResetStrategy.EARLIEST); + + reset(); + } + + // reset this mock restore consumer for a state store registration + public void reset() { + assignedPartition = null; + seekOffset = -1L; + endOffset = 0L; + recordBuffer.clear(); + } + + // buffer a record (we cannot use addRecord because we need to add records before assigning a partition) + public void bufferRecord(ConsumerRecord record) { + recordBuffer.add( + new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), 0L, + TimestampType.CREATE_TIME, 0L, 0, 0, + serializer.serialize(record.topic(), record.key()), + serializer.serialize(record.topic(), record.value()))); + endOffset = record.offset(); + + super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); + } + + @Override + public synchronized void assign(Collection partitions) { + int numPartitions = partitions.size(); + if (numPartitions > 1) + throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); + + if (numPartitions == 1) { + if (assignedPartition != null) + throw new IllegalStateException("RestoreConsumer: partition already assigned"); + assignedPartition = partitions.iterator().next(); + + // set the beginning offset to 0 + // NOTE: this is users responsible to set the initial lEO. + super.updateBeginningOffsets(Collections.singletonMap(assignedPartition, 0L)); + } + + super.assign(partitions); + } + + @Override + public ConsumerRecords poll(long timeout) { + // add buffered records to MockConsumer + for (ConsumerRecord record : recordBuffer) { + super.addRecord(record); + } + recordBuffer.clear(); + + ConsumerRecords records = super.poll(timeout); + + // set the current offset + Iterable> partitionRecords = records.records(assignedPartition); + for (ConsumerRecord record : partitionRecords) { + currentOffset = record.offset(); + } + + return records; + } + + @Override + public synchronized long position(TopicPartition partition) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: unassigned partition"); + + return currentOffset; + } + + @Override + public synchronized void seek(TopicPartition partition, long offset) { + if (offset < 0) + throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); + + if (seekOffset >= 0) + throw new IllegalStateException("RestoreConsumer: offset already seeked"); + + seekOffset = offset; + currentOffset = offset; + super.seek(partition, offset); + } + + @Override + public synchronized void seekToBeginning(Collection partitions) { + if (partitions.size() != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + currentOffset = 0L; + } + + + @Override + public Map endOffsets(final Collection partitions) { + if (partitions.size() != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + currentOffset = endOffset; + return super.endOffsets(partitions); + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index ac8933dcb6aac..842f30cd3edb3 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -56,6 +56,7 @@ import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.RecordCollectorImpl; import org.apache.kafka.streams.processor.internals.StateDirectory; +import org.apache.kafka.streams.processor.internals.StoreChangelogReader; import org.apache.kafka.streams.processor.internals.StreamTask; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.state.KeyValueStore; @@ -222,7 +223,7 @@ public List partitionsFor(String topic) { partitionsByTopic.values(), topology, consumer, - restoreStateConsumer, + new StoreChangelogReader(restoreStateConsumer, Time.SYSTEM, 5000), config, streamsMetrics, stateDirectory, cache, From 0483a0b0b70e1ec7cd12c17f622f91ebc3932fc4 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 24 Feb 2017 11:04:37 -0800 Subject: [PATCH 034/101] MINOR: Fix transient failure of testCannotSendToInternalTopic MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It’s a simple matter of creating the internal topic before trying to send to it. Otherwise, we could get an `UnknownTopicOrPartitionException` in some cases. Without the change, I could reproduce a failure in less than 5 runs. With the change, 30 consecutive runs succeeded. Author: Ismael Juma Reviewers: Apurva Mehta , Jason Gustafson Closes #2584 from ijuma/test-cannot-send-to-internal-topic-transient-failure --- .../kafka/api/IntegrationTestHarness.scala | 7 +-- .../api/ProducerFailureHandlingTest.scala | 15 +++--- .../scala/unit/kafka/utils/TestUtils.scala | 51 +++++++++++-------- 3 files changed, 39 insertions(+), 34 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 10baa42828c34..46465e8f28cd9 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -75,12 +75,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { consumers += createNewConsumer } - // create the consumer offset topic - TestUtils.createTopic(zkUtils, Topic.GroupMetadataTopicName, - serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, - serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, - servers, - servers.head.groupCoordinator.offsetsTopicConfigs) + TestUtils.createOffsetsTopic(zkUtils, servers) } def createNewProducer: KafkaProducer[Array[Byte], Array[Byte]] = { diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 5385bbe7b8dce..ce81cae10a021 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -123,7 +123,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { val record = new ProducerRecord(topic10, null, "key".getBytes, new Array[Byte](maxMessageSize - 50)) val recordMetadata = producer3.send(record).get - assertEquals(topic10, recordMetadata.topic()) + assertEquals(topic10, recordMetadata.topic) } /** This should succeed as the replica fetcher thread can handle oversized messages since KIP-74 */ @@ -176,8 +176,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { } /** - * Send with invalid partition id should throw KafkaException when partition is higher than the - * upper bound of partitions and IllegalArgumentException when partition is negative + * Send with invalid partition id should throw KafkaException when partition is higher than the upper bound of + * partitions. */ @Test def testInvalidPartition() { @@ -185,14 +185,14 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers) // create a record with incorrect partition id (higher than the number of partitions), send should fail - val higherRecord = new ProducerRecord[Array[Byte], Array[Byte]](topic1, 1, "key".getBytes, "value".getBytes) + val higherRecord = new ProducerRecord(topic1, 1, "key".getBytes, "value".getBytes) intercept[KafkaException] { producer1.send(higherRecord) } } /** - * The send call after producer closed should throw KafkaException cased by IllegalStateException + * The send call after producer closed should throw IllegalStateException */ @Test def testSendAfterClosed() { @@ -218,14 +218,13 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { producer3.close() producer3.send(record) } - - // re-close producer is fine } @Test def testCannotSendToInternalTopic() { + TestUtils.createOffsetsTopic(zkUtils, servers) val thrown = intercept[ExecutionException] { - producer2.send(new ProducerRecord[Array[Byte],Array[Byte]](Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get + producer2.send(new ProducerRecord(Topic.GroupMetadataTopicName, "test".getBytes, "test".getBytes)).get } assertTrue("Unexpected exception while sending to an invalid topic " + thrown.getCause, thrown.getCause.isInstanceOf[InvalidTopicException]) } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index c16618b8abaed..4f6a204b5d108 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -20,42 +20,40 @@ package kafka.utils import java.io._ import java.nio._ import java.nio.channels._ -import java.util.concurrent.{Callable, Executors, TimeUnit} -import java.util.Properties +import java.nio.charset.Charset import java.security.cert.X509Certificate +import java.util.Properties +import java.util.concurrent.{Callable, Executors, TimeUnit} import javax.net.ssl.X509TrustManager -import charset.Charset -import kafka.security.auth.{Acl, Authorizer, Resource} -import org.apache.kafka.common.protocol.SecurityProtocol -import org.apache.kafka.common.utils.Utils._ -import org.apache.kafka.test.TestSslUtils - -import scala.collection.mutable.{ArrayBuffer, ListBuffer} -import kafka.server._ -import kafka.producer._ -import kafka.message._ +import kafka.admin.AdminUtils import kafka.api._ import kafka.cluster.{Broker, EndPoint} +import kafka.common.{Topic, TopicAndPartition} import kafka.consumer.{ConsumerConfig, ConsumerTimeoutException, KafkaStream} -import kafka.serializer.{DefaultEncoder, Encoder, StringEncoder} -import kafka.common.TopicAndPartition -import kafka.admin.AdminUtils import kafka.log._ +import kafka.message._ +import kafka.producer._ +import kafka.security.auth.{Acl, Authorizer, Resource} +import kafka.serializer.{DefaultEncoder, Encoder, StringEncoder} +import kafka.server._ import kafka.utils.ZkUtils._ -import org.junit.Assert._ -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor} import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.{ListenerName, Mode} +import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.record._ import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer} import org.apache.kafka.common.utils.Time -import org.apache.kafka.test.{TestUtils => JTestUtils} +import org.apache.kafka.common.utils.Utils._ +import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} +import org.junit.Assert._ -import scala.collection.Map import scala.collection.JavaConverters._ +import scala.collection.Map +import scala.collection.mutable.{ArrayBuffer, ListBuffer} /** * Utility functions to help with testing @@ -270,6 +268,19 @@ object TestUtils extends Logging { }.toMap } + /** + * Create the consumer offsets/group metadata topic and wait until the leader is elected and metadata is propagated + * to all brokers. + */ + def createOffsetsTopic(zkUtils: ZkUtils, servers: Seq[KafkaServer]): Unit = { + val server = servers.head + createTopic(zkUtils, Topic.GroupMetadataTopicName, + server.config.getInt(KafkaConfig.OffsetsTopicPartitionsProp), + server.config.getShort(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, + servers, + server.groupCoordinator.offsetsTopicConfigs) + } + /** * Create a test config for a consumer */ From 43524442dc10c5dc731248674eb1a811287e88f7 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 24 Feb 2017 14:37:18 -0800 Subject: [PATCH 035/101] MINOR: Fixed Non-Final Close Method + its Duplication Author: Armin Braun Reviewers: Ismael Juma , Jason Gustafson Closes #2582 from original-brownbear/cleanup-nonfinal-close --- .../clients/consumer/internals/AbstractCoordinator.java | 2 +- .../kafka/connect/runtime/distributed/WorkerCoordinator.java | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index d36aac9f4e986..33061487966d1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -661,7 +661,7 @@ protected synchronized void requestRejoin() { * Close the coordinator, waiting if needed to send LeaveGroup. */ @Override - public synchronized void close() { + public final void close() { close(0); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java index 58525c5b0f08c..2d6af946dfb29 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java @@ -289,11 +289,6 @@ public String memberId() { return JoinGroupRequest.UNKNOWN_MEMBER_ID; } - @Override - public void close() { - super.close(); - } - private boolean isLeader() { return assignmentSnapshot != null && memberId().equals(assignmentSnapshot.leader()); } From b917af19014c6a8860d6523d24c87783c51bab4e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 24 Feb 2017 15:12:32 -0800 Subject: [PATCH 036/101] MINOR: Make it impossible to invoke `Request.body` without an explicit type parameter Author: Ismael Juma Reviewers: Jason Gustafson Closes #2579 from ijuma/safer-body --- .../scala/kafka/network/RequestChannel.scala | 8 ++-- .../main/scala/kafka/utils/NotNothing.scala | 41 +++++++++++++++++++ 2 files changed, 45 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/kafka/utils/NotNothing.scala diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 677f9b4fdbdc3..ad7a3fd5e4173 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -26,7 +26,7 @@ import com.yammer.metrics.core.Gauge import kafka.api.{ControlledShutdownRequest, RequestOrResponse} import kafka.metrics.KafkaMetricsGroup import kafka.server.QuotaId -import kafka.utils.Logging +import kafka.utils.{Logging, NotNothing} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.InvalidRequestException import org.apache.kafka.common.network.{ListenerName, Send} @@ -37,7 +37,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Time import org.apache.log4j.Logger -import scala.reflect.{classTag, ClassTag} +import scala.reflect.{ClassTag, classTag} object RequestChannel extends Logging { val AllDone = Request(processor = 1, connectionId = "2", Session(KafkaPrincipal.ANONYMOUS, InetAddress.getLocalHost), @@ -111,11 +111,11 @@ object RequestChannel extends Logging { s"$header -- ${body[AbstractRequest]}" } - def body[T <: AbstractRequest : ClassTag] = { + def body[T <: AbstractRequest](implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { bodyAndSize.request match { case r: T => r case r => - throw new ClassCastException(s"Expected request with type ${classTag[T].runtimeClass}, but found ${r.getClass}") + throw new ClassCastException(s"Expected request with type ${classTag.runtimeClass}, but found ${r.getClass}") } } diff --git a/core/src/main/scala/kafka/utils/NotNothing.scala b/core/src/main/scala/kafka/utils/NotNothing.scala new file mode 100644 index 0000000000000..aee345e75b51a --- /dev/null +++ b/core/src/main/scala/kafka/utils/NotNothing.scala @@ -0,0 +1,41 @@ +/** + * 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. + */ +@implicitNotFound("Unable to infer type parameter, please provide it explicitly.") +trait NotNothing[T] + +object NotNothing { + private val evidence: NotNothing[Any] = new Object with NotNothing[Any] + + implicit def notNothingEvidence[T](implicit n: T =:= T): NotNothing[T] = evidence.asInstanceOf[NotNothing[T]] +} From 3749832637b58b291a45076658feb8cd6e830247 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 24 Feb 2017 15:49:13 -0800 Subject: [PATCH 037/101] MINOR: Ensure consumer calls poll() if requests are outstanding Author: Jason Gustafson Reviewers: Onur Karaman , Ismael Juma Closes #2596 from hachikuji/ensure-poll-with-inflight-requests --- .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index edb8dc18f8e5f..3a46bd7231467 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1000,9 +1000,8 @@ public ConsumerRecords poll(long timeout) { // // NOTE: since the consumed position has already been updated, we must not allow // wakeups or any other errors to be triggered prior to returning the fetched records. - if (fetcher.sendFetches() > 0) { + if (fetcher.sendFetches() > 0 || client.pendingRequestCount() > 0) client.pollNoWakeup(); - } if (this.interceptors == null) return new ConsumerRecords<>(records); From 5b22b53f0e581e3239ca6bbeceec6c01ce0f8ce0 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 24 Feb 2017 16:41:51 -0800 Subject: [PATCH 038/101] MINOR: Fix potential integer overflow and String.format issue Author: Ismael Juma Reviewers: Apurva Mehta , Jason Gustafson Closes #2585 from ijuma/overflow-and-format-fixes --- .../producer/internals/BufferPool.java | 24 ++++++++++----- .../SaslClientAuthenticator.java | 2 +- .../producer/internals/BufferPoolTest.java | 30 +++++++++++++++++++ 3 files changed, 48 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index 077215c0f3589..92d59d9b82568 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java @@ -41,7 +41,7 @@ * buffers are deallocated. * */ -public final class BufferPool { +public class BufferPool { private final long totalMemory; private final int poolableSize; @@ -65,8 +65,8 @@ public final class BufferPool { public BufferPool(long memory, int poolableSize, Metrics metrics, Time time, String metricGrpName) { this.poolableSize = poolableSize; this.lock = new ReentrantLock(); - this.free = new ArrayDeque(); - this.waiters = new ArrayDeque(); + this.free = new ArrayDeque<>(); + this.waiters = new ArrayDeque<>(); this.totalMemory = memory; this.availableMemory = memory; this.metrics = metrics; @@ -104,14 +104,14 @@ public ByteBuffer allocate(int size, long maxTimeToBlockMs) throws InterruptedEx // now check if the request is immediately satisfiable with the // memory on hand or if we need to block - int freeListSize = this.free.size() * this.poolableSize; + int freeListSize = freeSize() * this.poolableSize; if (this.availableMemory + freeListSize >= size) { // we have enough unallocated or pooled memory to immediately // satisfy the request freeUp(size); this.availableMemory -= size; lock.unlock(); - return ByteBuffer.allocate(size); + return allocateByteBuffer(size); } else { // we are out of memory and will have to block int accumulated = 0; @@ -174,7 +174,7 @@ public ByteBuffer allocate(int size, long maxTimeToBlockMs) throws InterruptedEx // unlock and return the buffer lock.unlock(); if (buffer == null) - return ByteBuffer.allocate(size); + return allocateByteBuffer(size); else return buffer; } @@ -184,6 +184,11 @@ public ByteBuffer allocate(int size, long maxTimeToBlockMs) throws InterruptedEx } } + // Protected for testing. + protected ByteBuffer allocateByteBuffer(int size) { + return ByteBuffer.allocate(size); + } + /** * Attempt to ensure we have at least the requested number of bytes of memory for allocation by deallocating pooled * buffers (if needed) @@ -228,12 +233,17 @@ public void deallocate(ByteBuffer buffer) { public long availableMemory() { lock.lock(); try { - return this.availableMemory + this.free.size() * this.poolableSize; + return this.availableMemory + freeSize() * (long) this.poolableSize; } finally { lock.unlock(); } } + // Protected for testing. + protected int freeSize() { + return this.free.size(); + } + /** * Get the unallocated memory (not in the free list or in use) */ diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java index 2b445e5cfc1eb..27a24e5bf09a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java @@ -335,7 +335,7 @@ private void handleSaslHandshakeResponse(SaslHandshakeResponse response) { throw new IllegalSaslStateException(String.format("Unexpected handshake request with client mechanism %s, enabled mechanisms are %s", mechanism, response.enabledMechanisms())); default: - throw new AuthenticationException(String.format("Unknown error code %d, client mechanism is %s, enabled mechanisms are %s", + throw new AuthenticationException(String.format("Unknown error code %s, client mechanism is %s, enabled mechanisms are %s", response.error(), mechanism, response.enabledMechanisms())); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java index 41ac4f0b1c593..41143d874d161 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java @@ -31,6 +31,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import static org.junit.Assert.assertNotEquals; @@ -268,6 +269,35 @@ public void testStressfulSituation() throws Exception { assertEquals(totalMemory, pool.availableMemory()); } + @Test + public void testLargeAvailableMemory() throws Exception { + long memory = 20_000_000_000L; + int poolableSize = 2_000_000_000; + final AtomicInteger freeSize = new AtomicInteger(0); + BufferPool pool = new BufferPool(memory, poolableSize, metrics, time, metricGroup) { + @Override + protected ByteBuffer allocateByteBuffer(int size) { + // Ignore size to avoid OOM due to large buffers + return ByteBuffer.allocate(0); + } + + @Override + protected int freeSize() { + return freeSize.get(); + } + }; + pool.allocate(poolableSize, 0); + assertEquals(18_000_000_000L, pool.availableMemory()); + pool.allocate(poolableSize, 0); + assertEquals(16_000_000_000L, pool.availableMemory()); + + // Emulate `deallocate` by increasing `freeSize` + freeSize.incrementAndGet(); + assertEquals(18_000_000_000L, pool.availableMemory()); + freeSize.incrementAndGet(); + assertEquals(20_000_000_000L, pool.availableMemory()); + } + public static class StressTestThread extends Thread { private final int iterations; private final BufferPool pool; From 5b682baad546e41fe7aa27be0865ea6adaca2d2a Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Sat, 25 Feb 2017 18:44:32 -0800 Subject: [PATCH 039/101] MINOR: Use API hyperlinks in 'Kafka Protocol Guide' to facilitate navigation Finding the protocol associated with an API key can be a challenge in the lengthy [web page](http://kafka.apache.org/protocol.html#protocol_api_keys). Adding hyperlinks would definitely help with that. Co-authored with imandhan. Author: Vahid Hashemian Reviewers: Ewen Cheslack-Postava Closes #2467 from vahidhashemian/minor/hyperlinks_in_kafka_protocol_guide --- .../main/java/org/apache/kafka/common/protocol/ApiKeys.java | 2 +- .../main/java/org/apache/kafka/common/protocol/Protocol.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 50d89f1e9f9c7..4e525df00d60d 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -136,7 +136,7 @@ private static String toHtml() { for (ApiKeys key : ApiKeys.values()) { b.append("

    \n"); b.append(""); b.append("
    "); - b.append(getConfigValue(def, headerName)); + b.append(getConfigValue(key, headerName)); b.append("
    "); - b.append(key.name); + b.append("" + key.name + ""); b.append(""); b.append(key.id); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index b04587fcb86df..032e00e0a158f 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -1176,10 +1176,11 @@ public static String toHtml() { for (ApiKeys key : ApiKeys.values()) { // Key b.append("
    "); + b.append(""); b.append(key.name); b.append(" API (Key: "); b.append(key.id); - b.append("):
    \n\n"); + b.append("):\n\n"); // Requests b.append("Requests:
    \n"); Schema[] requests = REQUESTS[key.id]; From 9260e8563c19cc1c5ac2d47ad05955774138036d Mon Sep 17 00:00:00 2001 From: Marco Ebert Date: Mon, 27 Feb 2017 11:48:50 +0000 Subject: [PATCH 040/101] KAFKA-4806; Prevent double logging of ConsumerConfig The consumer properties get logged twice since two instances of ConsumerConfig are created during creation of KafkaConsumer. I added a constructor of ConsumerConfig accepting the boolean parameter doLog which is already passable in AbstractConfig and set it to false during the second ConsumerConfig creating in the KafkaConsumer constructor. Author: Marco Ebert Reviewers: Ismael Juma Closes #2600 from Gacko/trunk --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 4 ++++ .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index ed809a9f401ad..1b335173b8884 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -426,6 +426,10 @@ public static Properties addDeserializerToConfig(Properties properties, super(CONFIG, props); } + ConsumerConfig(Map props, boolean doLog) { + super(CONFIG, props, doLog); + } + public static Set configNames() { return CONFIG.names(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 3a46bd7231467..a90303632a436 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -630,7 +630,7 @@ private KafkaConsumer(ConsumerConfig config, // load interceptors and make sure they get clientId Map userProvidedConfigs = config.originals(); userProvidedConfigs.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId); - List> interceptorList = (List) (new ConsumerConfig(userProvidedConfigs)).getConfiguredInstances(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, + List> interceptorList = (List) (new ConsumerConfig(userProvidedConfigs, false)).getConfiguredInstances(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, ConsumerInterceptor.class); this.interceptors = interceptorList.isEmpty() ? null : new ConsumerInterceptors<>(interceptorList); if (keyDeserializer == null) { From 9231cc439e58d6ac252ca960e15c703366e32e1a Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Mon, 27 Feb 2017 11:19:09 -0800 Subject: [PATCH 041/101] KAFKA-4744: Increased timeout for bounce test Author: Eno Thereska Reviewers: Ismael Juma, Matthias J. Sax, Guozhang Wang Closes #2601 from enothereska/KAFKA-4744-bounce --- tests/kafkatest/services/streams.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index f231d421fac56..1e1c676151879 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -77,7 +77,7 @@ def stop_node(self, node, clean_shutdown=True): node.account.signal(pid, sig, allow_fail=True) if clean_shutdown: for pid in pids: - wait_until(lambda: not node.account.alive(pid), timeout_sec=60, err_msg="Streams Test process on " + str(node.account) + " took too long to exit") + wait_until(lambda: not node.account.alive(pid), timeout_sec=120, err_msg="Streams Test process on " + str(node.account) + " took too long to exit") node.account.ssh("rm -f " + self.PID_FILE, allow_fail=False) From 8e6fbe8fed592e7cc15731a0827c350794413767 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 28 Feb 2017 11:49:16 -0800 Subject: [PATCH 042/101] MINOR: Make asJsonSchema() and asConnectSchema() methods public Want to use these methods in an external project. Author: Chris Egerton Reviewers: Ewen Cheslack-Postava Closes #2610 from C0urante/public-json-schema-conversion --- .../java/org/apache/kafka/connect/json/JsonConverter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index dd83886fcf1fd..c5d4858e4da61 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -332,7 +332,7 @@ private SchemaAndValue jsonToConnect(JsonNode jsonValue) { return new SchemaAndValue(schema, convertToConnect(schema, jsonValue.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME))); } - private ObjectNode asJsonSchema(Schema schema) { + public ObjectNode asJsonSchema(Schema schema) { if (schema == null) return null; @@ -413,7 +413,7 @@ private ObjectNode asJsonSchema(Schema schema) { } - private Schema asConnectSchema(JsonNode jsonSchema) { + public Schema asConnectSchema(JsonNode jsonSchema) { if (jsonSchema.isNull()) return null; From f3fab2e476cf5101ba6ae68e4d61ae9ab957b26c Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Tue, 28 Feb 2017 12:21:46 -0800 Subject: [PATCH 043/101] KAFKA-4809: docker/run_tests.sh should set up /opt/kafka-dev to be the source directory MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …0.x and not 0.8 Author: Colin P. Mccabe Reviewers: Ewen Cheslack-Postava Closes #2602 from cmccabe/KAFKA-4809 --- tests/docker/Dockerfile | 20 +++++++------------ tests/docker/run_tests.sh | 14 ++----------- .../sanity_checks/test_verifiable_producer.py | 4 +++- 3 files changed, 12 insertions(+), 26 deletions(-) diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 82e2e55f4320a..3a3df3242259a 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -16,22 +16,16 @@ FROM openjdk:8 MAINTAINER Apache Kafka dev@kafka.apache.org -RUN apt update -RUN apt install -y unzip wget curl jq coreutils openssh-server net-tools vim openjdk-8-jdk python-pip python-dev libffi-dev libssl-dev -RUN pip install -U pip -RUN pip install --upgrade cffi -RUN pip install ducktape==0.6.0 - -VOLUME ["/kafka_src"] - +VOLUME ["/opt/kafka-dev"] ENV MIRROR="http://apache.cs.utah.edu/" -RUN wget -q "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" -O "/tmp/kafka_2.10-0.8.2.2.tgz" && tar xfz /tmp/kafka_2.10-0.8.2.2.tgz -C /opt && mv "/opt/kafka_2.10-0.8.2.2" "/opt/kafka-0.8.2.2" -RUN wget -q "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" -O "/tmp/kafka_2.10-0.9.0.1.tgz" && tar xfz /tmp/kafka_2.10-0.9.0.1.tgz -C /opt && mv "/opt/kafka_2.10-0.9.0.1" "/opt/kafka-0.9.0.1" -RUN wget -q "${MIRROR}kafka/0.10.0.1/kafka_2.10-0.10.0.1.tgz" -O "/tmp/kafka_2.10-0.10.0.1.tgz" && tar xfz /tmp/kafka_2.10-0.10.0.1.tgz -C /opt && mv "/opt/kafka_2.10-0.10.0.1" "/opt/kafka-0.10.0.1" -RUN wget -q "${MIRROR}kafka/0.10.1.1/kafka_2.10-0.10.1.1.tgz" -O "/tmp/kafka_2.10-0.10.1.1.tgz" && tar xfz /tmp/kafka_2.10-0.10.1.1.tgz -C /opt && mv "/opt/kafka_2.10-0.10.1.1" "/opt/kafka-0.10.1.1" -RUN rm /tmp/kafka_*.tgz ADD ssh /root/.ssh RUN chmod 600 /root/.ssh/id_rsa +RUN apt update && apt install -y unzip wget curl jq coreutils openssh-server net-tools vim openjdk-8-jdk python-pip python-dev libffi-dev libssl-dev +RUN pip install -U pip && pip install --upgrade cffi ducktape==0.6.0 +RUN mkdir -p "/opt/kafka_2.10-0.8.2.2" && curl "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.8.2.2" +RUN mkdir -p "/opt/kafka_2.10-0.9.0.1" && curl "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.9.0.1" +RUN mkdir -p "/opt/kafka_2.10-0.10.0.1" && curl "${MIRROR}kafka/0.10.0.1/kafka_2.10-0.10.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.10.0.1" +RUN mkdir -p "/opt/kafka_2.10-0.10.1.1" && curl "${MIRROR}kafka/0.10.1.1/kafka_2.10-0.10.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.10.1.1" CMD service ssh start && tail -f /dev/null diff --git a/tests/docker/run_tests.sh b/tests/docker/run_tests.sh index 0e5378a8f26b8..cc381cc33c2e4 100755 --- a/tests/docker/run_tests.sh +++ b/tests/docker/run_tests.sh @@ -46,7 +46,7 @@ fi echo "Using kafka image: ${KAFKA_IMAGE}" docker inspect ${KAFKA_IMAGE} for i in $(seq -w 1 ${KAFKA_NUM_CONTAINERS}); do - docker run -d -t --name knode${i} --network knw -v ${KAFKA_SRC}:/kafka_src ${KAFKA_IMAGE} + docker run -d -t --name knode${i} --network knw -v ${KAFKA_SRC}:/opt/kafka-dev ${KAFKA_IMAGE} done docker info @@ -55,18 +55,8 @@ docker network inspect knw for i in $(seq -w 1 ${KAFKA_NUM_CONTAINERS}); do echo knode${i} - docker exec knode${i} bash -c "(tar xfz /kafka_src/core/build/distributions/kafka_*SNAPSHOT.tgz -C /opt || echo missing kafka tgz did you build kafka tarball) && mv /opt/kafka*SNAPSHOT /opt/kafka-dev && ls -l /opt" - docker exec knode01 bash -c "ssh knode$i hostname" -done - -# hack to copy test dependencies -# this is required for running MiniKDC -(cd ${KAFKA_SRC} && ./gradlew copyDependantTestLibs) -for i in $(seq -w 1 ${KAFKA_NUM_CONTAINERS}); do - echo knode${i} - docker exec knode${i} bash -c "cp /kafka_src/core/build/dependant-testlibs/* /opt/kafka-dev/libs/" docker exec knode01 bash -c "ssh knode$i hostname" done bash tests/cluster_file_generator.sh > tests/cluster_file.json -docker exec knode01 bash -c "cd /kafka_src; ducktape ${_DUCKTAPE_OPTIONS} --cluster-file tests/cluster_file.json ${TC_PATHS:-tests/kafkatest/tests}" +docker exec knode01 bash -c "cd /opt/kafka-dev; ducktape ${_DUCKTAPE_OPTIONS} --cluster-file tests/cluster_file.json ${TC_PATHS:-tests/kafkatest/tests}" diff --git a/tests/kafkatest/sanity_checks/test_verifiable_producer.py b/tests/kafkatest/sanity_checks/test_verifiable_producer.py index b4ca4e94f4b56..be1057471948d 100644 --- a/tests/kafkatest/sanity_checks/test_verifiable_producer.py +++ b/tests/kafkatest/sanity_checks/test_verifiable_producer.py @@ -23,7 +23,7 @@ from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.services.zookeeper import ZookeeperService from kafkatest.utils import is_version -from kafkatest.version import LATEST_0_8_2, LATEST_0_9, DEV_BRANCH, KafkaVersion +from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, LATEST_0_10_1, DEV_BRANCH, KafkaVersion class TestVerifiableProducer(Test): @@ -48,6 +48,8 @@ def setUp(self): @cluster(num_nodes=3) @parametrize(producer_version=str(LATEST_0_8_2)) @parametrize(producer_version=str(LATEST_0_9)) + @parametrize(producer_version=str(LATEST_0_10_0)) + @parametrize(producer_version=str(LATEST_0_10_1)) @parametrize(producer_version=str(DEV_BRANCH)) def test_simple_run(self, producer_version=DEV_BRANCH): """ From d0e436c471ba4122ddcc0f7a1624546f97c4a517 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 28 Feb 2017 12:35:04 -0800 Subject: [PATCH 044/101] MINOR: improve license header check by providing head file instead of (prefix) header regex Author: Matthias J. Sax Reviewers: Jason Gustafson , Ewen Cheslack-Postava Closes #2303 from mjsax/licenseHeader --- checkstyle/checkstyle.xml | 4 +-- checkstyle/java.header | 16 +++++++++ .../apache/kafka/clients/ClientRequest.java | 26 ++++++++------- .../apache/kafka/clients/ClientResponse.java | 26 ++++++++------- .../org/apache/kafka/clients/ClientUtils.java | 22 ++++++++----- .../clients/ClusterConnectionStates.java | 26 ++++++++------- .../kafka/clients/CommonClientConfigs.java | 23 +++++++------ .../apache/kafka/clients/ConnectionState.java | 26 ++++++++------- .../kafka/clients/InFlightRequests.java | 26 ++++++++------- .../org/apache/kafka/clients/KafkaClient.java | 26 ++++++++------- .../kafka/clients/ManualMetadataUpdater.java | 23 +++++++------ .../org/apache/kafka/clients/Metadata.java | 22 ++++++++----- .../apache/kafka/clients/MetadataUpdater.java | 23 +++++++------ .../apache/kafka/clients/NetworkClient.java | 22 ++++++++----- .../apache/kafka/clients/NodeApiVersions.java | 22 ++++++++----- .../clients/RequestCompletionHandler.java | 26 ++++++++------- .../consumer/CommitFailedException.java | 6 ++-- .../kafka/clients/consumer/Consumer.java | 22 ++++++++----- .../clients/consumer/ConsumerConfig.java | 22 ++++++++----- .../clients/consumer/ConsumerInterceptor.java | 22 ++++++++----- .../consumer/ConsumerRebalanceListener.java | 22 ++++++++----- .../clients/consumer/ConsumerRecord.java | 26 ++++++++------- .../clients/consumer/ConsumerRecords.java | 26 ++++++++------- .../consumer/InvalidOffsetException.java | 22 ++++++++----- .../kafka/clients/consumer/KafkaConsumer.java | 22 ++++++++----- .../kafka/clients/consumer/MockConsumer.java | 22 ++++++++----- .../NoOffsetForPartitionException.java | 27 ++++++++------- .../clients/consumer/OffsetAndMetadata.java | 22 ++++++++----- .../clients/consumer/OffsetAndTimestamp.java | 6 ++-- .../consumer/OffsetCommitCallback.java | 22 ++++++++----- .../consumer/OffsetOutOfRangeException.java | 26 ++++++++------- .../clients/consumer/OffsetResetStrategy.java | 22 ++++++++----- .../kafka/clients/consumer/RangeAssignor.java | 12 +++---- .../RetriableCommitFailedException.java | 6 ++-- .../clients/consumer/RoundRobinAssignor.java | 26 ++++++++------- .../internals/AbstractCoordinator.java | 26 ++++++++------- .../internals/AbstractPartitionAssignor.java | 26 ++++++++------- .../internals/ConsumerCoordinator.java | 22 ++++++++----- .../internals/ConsumerInterceptors.java | 22 ++++++++----- .../internals/ConsumerNetworkClient.java | 22 ++++++++----- .../consumer/internals/ConsumerProtocol.java | 6 ++-- .../clients/consumer/internals/Fetcher.java | 23 +++++++------ .../clients/consumer/internals/Heartbeat.java | 26 ++++++++------- .../NoAvailableBrokersException.java | 22 ++++++++----- .../NoOpConsumerRebalanceListener.java | 27 ++++++++------- .../consumer/internals/PartitionAssignor.java | 22 ++++++++----- .../consumer/internals/RequestFuture.java | 22 ++++++++----- .../internals/RequestFutureAdapter.java | 22 ++++++++----- .../internals/RequestFutureListener.java | 22 ++++++++----- .../internals/StaleMetadataException.java | 22 ++++++++----- .../consumer/internals/SubscriptionState.java | 22 ++++++++----- .../producer/BufferExhaustedException.java | 6 ++-- .../kafka/clients/producer/Callback.java | 6 ++-- .../kafka/clients/producer/KafkaProducer.java | 22 ++++++++----- .../kafka/clients/producer/MockProducer.java | 6 ++-- .../kafka/clients/producer/Partitioner.java | 7 ++-- .../kafka/clients/producer/Producer.java | 6 ++-- .../clients/producer/ProducerConfig.java | 22 ++++++++----- .../clients/producer/ProducerInterceptor.java | 6 ++-- .../clients/producer/ProducerRecord.java | 26 ++++++++------- .../clients/producer/RecordMetadata.java | 6 ++-- .../producer/internals/BufferPool.java | 6 ++-- .../internals/DefaultPartitioner.java | 6 ++-- .../internals/ErrorLoggingCallback.java | 23 +++++++------ .../internals/FutureRecordMetadata.java | 26 ++++++++------- .../internals/ProduceRequestResult.java | 6 ++-- .../internals/ProducerInterceptors.java | 6 ++-- .../producer/internals/RecordAccumulator.java | 26 ++++++++------- .../producer/internals/RecordBatch.java | 26 ++++++++------- .../clients/producer/internals/Sender.java | 26 ++++++++------- .../java/org/apache/kafka/common/Cluster.java | 26 ++++++++------- .../apache/kafka/common/ClusterResource.java | 7 ++-- .../kafka/common/ClusterResourceListener.java | 23 +++++++------ .../org/apache/kafka/common/Configurable.java | 6 ++-- .../apache/kafka/common/KafkaException.java | 6 ++-- .../java/org/apache/kafka/common/Metric.java | 6 ++-- .../org/apache/kafka/common/MetricName.java | 24 ++++++++------ .../java/org/apache/kafka/common/Node.java | 22 ++++++++----- .../apache/kafka/common/PartitionInfo.java | 26 ++++++++------- .../apache/kafka/common/TopicPartition.java | 6 ++-- .../common/annotation/InterfaceStability.java | 22 ++++++++----- .../org/apache/kafka/common/cache/Cache.java | 9 +++-- .../apache/kafka/common/cache/LRUCache.java | 9 +++-- .../kafka/common/cache/SynchronizedCache.java | 9 +++-- .../kafka/common/config/AbstractConfig.java | 22 ++++++++----- .../apache/kafka/common/config/Config.java | 22 ++++++++----- .../apache/kafka/common/config/ConfigDef.java | 22 ++++++++----- .../kafka/common/config/ConfigException.java | 6 ++-- .../kafka/common/config/ConfigValue.java | 15 ++++----- .../kafka/common/config/SaslConfigs.java | 23 +++++++------ .../kafka/common/config/SslConfigs.java | 23 +++++++------ .../kafka/common/config/types/Password.java | 9 +++-- .../kafka/common/errors/ApiException.java | 6 ++-- .../errors/AuthenticationException.java | 22 ++++++++----- .../common/errors/AuthorizationException.java | 22 ++++++++----- .../errors/BrokerNotAvailableException.java | 7 ++-- .../errors/ClusterAuthorizationException.java | 22 ++++++++----- .../errors/ControllerMovedException.java | 7 ++-- .../common/errors/CorruptRecordException.java | 22 ++++++++----- .../common/errors/DisconnectException.java | 22 ++++++++----- .../errors/GroupAuthorizationException.java | 22 ++++++++----- ...GroupCoordinatorNotAvailableException.java | 23 +++++++------ .../errors/GroupLoadInProgressException.java | 23 +++++++------ .../errors/IllegalGenerationException.java | 22 ++++++++----- .../errors/IllegalSaslStateException.java | 22 ++++++++----- .../InconsistentGroupProtocolException.java | 22 ++++++++----- .../common/errors/InterruptException.java | 26 ++++++++------- .../InvalidCommitOffsetSizeException.java | 22 ++++++++----- .../errors/InvalidConfigurationException.java | 8 ++--- .../errors/InvalidFetchSizeException.java | 22 ++++++++----- .../errors/InvalidGroupIdException.java | 22 ++++++++----- .../errors/InvalidMetadataException.java | 27 ++++++++------- .../common/errors/InvalidOffsetException.java | 7 ++-- .../errors/InvalidPartitionsException.java | 8 ++--- .../InvalidReplicaAssignmentException.java | 8 ++--- .../InvalidReplicationFactorException.java | 8 ++--- .../errors/InvalidRequestException.java | 8 ++--- .../errors/InvalidRequiredAcksException.java | 8 ++--- .../InvalidSessionTimeoutException.java | 22 ++++++++----- .../errors/InvalidTimestampException.java | 7 ++-- .../common/errors/InvalidTopicException.java | 26 ++++++++------- .../errors/LeaderNotAvailableException.java | 22 ++++++++----- .../kafka/common/errors/NetworkException.java | 26 ++++++++------- .../common/errors/NotControllerException.java | 23 +++++++------ .../NotCoordinatorForGroupException.java | 23 +++++++------ ...NotEnoughReplicasAfterAppendException.java | 8 ++--- .../errors/NotEnoughReplicasException.java | 8 ++--- .../NotLeaderForPartitionException.java | 26 ++++++++------- .../common/errors/OffsetMetadataTooLarge.java | 22 ++++++++----- .../errors/OffsetOutOfRangeException.java | 7 ++-- .../errors/PolicyViolationException.java | 7 ++-- .../errors/RebalanceInProgressException.java | 6 ++-- .../errors/RecordBatchTooLargeException.java | 22 ++++++++----- .../errors/RecordTooLargeException.java | 26 ++++++++------- .../errors/ReplicaNotAvailableException.java | 7 ++-- .../common/errors/RetriableException.java | 26 ++++++++------- .../common/errors/SerializationException.java | 22 ++++++++----- .../kafka/common/errors/TimeoutException.java | 26 ++++++++------- .../errors/TopicAuthorizationException.java | 22 ++++++++----- .../common/errors/TopicExistsException.java | 8 ++--- .../errors/UnknownMemberIdException.java | 22 ++++++++----- .../common/errors/UnknownServerException.java | 26 ++++++++------- .../UnknownTopicOrPartitionException.java | 26 ++++++++------- .../UnsupportedForMessageFormatException.java | 22 ++++++++----- .../UnsupportedSaslMechanismException.java | 22 ++++++++----- .../errors/UnsupportedVersionException.java | 22 ++++++++----- .../kafka/common/errors/WakeupException.java | 22 ++++++++----- .../internals/ClusterResourceListeners.java | 7 ++-- .../common/internals/FatalExitError.java | 22 ++++++++----- .../common/internals/PartitionStates.java | 7 ++-- .../kafka/common/metrics/CompoundStat.java | 6 ++-- .../kafka/common/metrics/JmxReporter.java | 26 ++++++++------- .../kafka/common/metrics/KafkaMetric.java | 6 ++-- .../kafka/common/metrics/Measurable.java | 26 ++++++++------- .../kafka/common/metrics/MeasurableStat.java | 6 ++-- .../kafka/common/metrics/MetricConfig.java | 6 ++-- .../apache/kafka/common/metrics/Metrics.java | 26 ++++++++------- .../kafka/common/metrics/MetricsReporter.java | 26 ++++++++------- .../apache/kafka/common/metrics/Quota.java | 6 ++-- .../metrics/QuotaViolationException.java | 6 ++-- .../apache/kafka/common/metrics/Sensor.java | 26 ++++++++------- .../org/apache/kafka/common/metrics/Stat.java | 6 ++-- .../kafka/common/metrics/stats/Avg.java | 26 ++++++++------- .../kafka/common/metrics/stats/Count.java | 26 ++++++++------- .../kafka/common/metrics/stats/Histogram.java | 6 ++-- .../kafka/common/metrics/stats/Max.java | 26 ++++++++------- .../kafka/common/metrics/stats/Min.java | 26 ++++++++------- .../common/metrics/stats/Percentile.java | 6 ++-- .../common/metrics/stats/Percentiles.java | 26 ++++++++------- .../kafka/common/metrics/stats/Rate.java | 26 ++++++++------- .../common/metrics/stats/SampledStat.java | 26 ++++++++------- .../common/metrics/stats/SimpleRate.java | 12 +++---- .../kafka/common/metrics/stats/Total.java | 26 ++++++++------- .../kafka/common/metrics/stats/Value.java | 22 ++++++++----- .../kafka/common/network/Authenticator.java | 7 ++-- .../common/network/ByteBufferReceive.java | 6 ++-- .../kafka/common/network/ByteBufferSend.java | 22 ++++++++----- .../kafka/common/network/ChannelBuilder.java | 22 ++++++++----- .../kafka/common/network/ChannelBuilders.java | 23 +++++++------ .../common/network/DefaultAuthenticator.java | 7 ++-- .../network/InvalidReceiveException.java | 8 ++--- .../kafka/common/network/KafkaChannel.java | 7 ++-- .../kafka/common/network/ListenerName.java | 8 ++--- .../org/apache/kafka/common/network/Mode.java | 6 ++-- .../kafka/common/network/MultiSend.java | 8 ++--- .../kafka/common/network/NetworkReceive.java | 22 ++++++++----- .../kafka/common/network/NetworkSend.java | 6 ++-- .../network/PlaintextChannelBuilder.java | 22 ++++++++----- .../network/PlaintextTransportLayer.java | 7 ++-- .../apache/kafka/common/network/Receive.java | 6 ++-- .../common/network/SaslChannelBuilder.java | 22 ++++++++----- .../kafka/common/network/Selectable.java | 22 ++++++++----- .../apache/kafka/common/network/Selector.java | 22 ++++++++----- .../org/apache/kafka/common/network/Send.java | 22 ++++++++----- .../common/network/SslChannelBuilder.java | 22 ++++++++----- .../common/network/SslTransportLayer.java | 7 ++-- .../kafka/common/network/TransportLayer.java | 7 ++-- .../kafka/common/network/TransportLayers.java | 7 ++-- .../apache/kafka/common/protocol/ApiKeys.java | 6 ++-- .../apache/kafka/common/protocol/Errors.java | 6 ++-- .../kafka/common/protocol/Protocol.java | 6 ++-- .../common/protocol/SecurityProtocol.java | 8 ++--- .../kafka/common/protocol/types/ArrayOf.java | 6 ++-- .../kafka/common/protocol/types/Field.java | 6 ++-- .../kafka/common/protocol/types/Schema.java | 26 ++++++++------- .../protocol/types/SchemaException.java | 6 ++-- .../kafka/common/protocol/types/Struct.java | 22 ++++++++----- .../kafka/common/protocol/types/Type.java | 6 ++-- .../kafka/common/record/AbstractRecords.java | 14 ++++---- .../common/record/ByteBufferInputStream.java | 6 ++-- .../record/ByteBufferLogInputStream.java | 14 ++++---- .../common/record/ByteBufferOutputStream.java | 6 ++-- .../kafka/common/record/CompressionType.java | 6 ++-- .../common/record/FileLogInputStream.java | 14 ++++---- .../kafka/common/record/FileRecords.java | 14 ++++---- .../common/record/InvalidRecordException.java | 6 ++-- .../record/KafkaLZ4BlockInputStream.java | 7 ++-- .../record/KafkaLZ4BlockOutputStream.java | 7 ++-- .../apache/kafka/common/record/LogEntry.java | 6 ++-- .../kafka/common/record/LogInputStream.java | 14 ++++---- .../kafka/common/record/MemoryRecords.java | 26 ++++++++------- .../common/record/MemoryRecordsBuilder.java | 6 ++-- .../apache/kafka/common/record/Record.java | 6 ++-- .../apache/kafka/common/record/Records.java | 6 ++-- .../kafka/common/record/RecordsIterator.java | 14 ++++---- .../kafka/common/record/TimestampType.java | 7 ++-- .../common/requests/AbstractRequest.java | 8 ++--- .../requests/AbstractRequestResponse.java | 22 ++++++++----- .../common/requests/AbstractResponse.java | 14 ++++---- .../common/requests/ApiVersionsRequest.java | 22 ++++++++----- .../common/requests/ApiVersionsResponse.java | 22 ++++++++----- .../requests/ControlledShutdownRequest.java | 22 ++++++++----- .../requests/ControlledShutdownResponse.java | 22 ++++++++----- .../common/requests/CreateTopicsRequest.java | 8 ++--- .../common/requests/CreateTopicsResponse.java | 8 ++--- .../common/requests/DeleteTopicsRequest.java | 8 ++--- .../common/requests/DeleteTopicsResponse.java | 8 ++--- .../requests/DescribeGroupsRequest.java | 22 ++++++++----- .../requests/DescribeGroupsResponse.java | 22 ++++++++----- .../kafka/common/requests/FetchRequest.java | 22 ++++++++----- .../kafka/common/requests/FetchResponse.java | 6 ++-- .../requests/GroupCoordinatorRequest.java | 22 ++++++++----- .../requests/GroupCoordinatorResponse.java | 22 ++++++++----- .../common/requests/HeartbeatRequest.java | 22 ++++++++----- .../common/requests/HeartbeatResponse.java | 22 ++++++++----- .../common/requests/JoinGroupRequest.java | 22 ++++++++----- .../common/requests/JoinGroupResponse.java | 22 ++++++++----- .../common/requests/LeaderAndIsrRequest.java | 7 ++-- .../common/requests/LeaderAndIsrResponse.java | 23 +++++++------ .../common/requests/LeaveGroupRequest.java | 22 ++++++++----- .../common/requests/LeaveGroupResponse.java | 22 ++++++++----- .../common/requests/ListGroupsRequest.java | 22 ++++++++----- .../common/requests/ListGroupsResponse.java | 22 ++++++++----- .../common/requests/ListOffsetRequest.java | 6 ++-- .../common/requests/ListOffsetResponse.java | 6 ++-- .../common/requests/MetadataRequest.java | 22 ++++++++----- .../common/requests/MetadataResponse.java | 22 ++++++++----- .../common/requests/OffsetCommitRequest.java | 22 ++++++++----- .../common/requests/OffsetCommitResponse.java | 22 ++++++++----- .../common/requests/OffsetFetchRequest.java | 22 ++++++++----- .../common/requests/OffsetFetchResponse.java | 22 ++++++++----- .../kafka/common/requests/PartitionState.java | 23 +++++++------ .../kafka/common/requests/ProduceRequest.java | 23 +++++++------ .../common/requests/ProduceResponse.java | 26 ++++++++------- .../kafka/common/requests/RecordsSend.java | 14 ++++---- .../kafka/common/requests/RequestAndSize.java | 6 ++-- .../kafka/common/requests/RequestHeader.java | 26 ++++++++------- .../kafka/common/requests/ResponseHeader.java | 6 ++-- .../common/requests/SaslHandshakeRequest.java | 18 +++++----- .../requests/SaslHandshakeResponse.java | 18 +++++----- .../common/requests/StopReplicaRequest.java | 23 +++++++------ .../common/requests/StopReplicaResponse.java | 23 +++++++------ .../common/requests/SyncGroupRequest.java | 6 ++-- .../common/requests/SyncGroupResponse.java | 6 ++-- .../requests/UpdateMetadataRequest.java | 23 +++++++------ .../requests/UpdateMetadataResponse.java | 23 +++++++------ .../kafka/common/security/JaasConfig.java | 7 ++-- .../kafka/common/security/JaasContext.java | 9 +++-- .../kafka/common/security/JaasUtils.java | 8 ++--- .../security/auth/AuthCallbackHandler.java | 7 ++-- .../auth/DefaultPrincipalBuilder.java | 7 ++-- .../common/security/auth/KafkaPrincipal.java | 12 +++---- .../kafka/common/security/auth/Login.java | 18 +++++----- .../security/auth/PrincipalBuilder.java | 7 ++-- .../security/authenticator/AbstractLogin.java | 18 +++++----- .../authenticator/CredentialCache.java | 18 +++++----- .../security/authenticator/DefaultLogin.java | 18 +++++----- .../security/authenticator/LoginManager.java | 18 +++++----- .../SaslClientAuthenticator.java | 18 +++++----- .../SaslClientCallbackHandler.java | 17 +++++----- .../SaslServerAuthenticator.java | 7 ++-- .../SaslServerCallbackHandler.java | 18 +++++----- .../security/kerberos/BadFormatString.java | 18 +++++----- .../security/kerberos/KerberosLogin.java | 18 +++++----- .../security/kerberos/KerberosName.java | 18 +++++----- .../security/kerberos/KerberosRule.java | 18 +++++----- .../security/kerberos/KerberosShortNamer.java | 18 +++++----- .../security/kerberos/NoMatchingRule.java | 18 +++++----- .../security/plain/PlainLoginModule.java | 18 +++++----- .../security/plain/PlainSaslServer.java | 18 +++++----- .../plain/PlainSaslServerProvider.java | 18 +++++----- .../security/scram/ScramCredential.java | 18 +++++----- .../scram/ScramCredentialCallback.java | 7 ++-- .../security/scram/ScramCredentialUtils.java | 20 +++++------ .../common/security/scram/ScramFormatter.java | 18 +++++----- .../security/scram/ScramLoginModule.java | 18 +++++----- .../common/security/scram/ScramMechanism.java | 18 +++++----- .../common/security/scram/ScramMessages.java | 18 +++++----- .../security/scram/ScramSaslClient.java | 18 +++++----- .../scram/ScramSaslClientProvider.java | 18 +++++----- .../security/scram/ScramSaslServer.java | 18 +++++----- .../scram/ScramSaslServerProvider.java | 18 +++++----- .../scram/ScramServerCallbackHandler.java | 7 ++-- .../kafka/common/security/ssl/SslFactory.java | 6 ++-- .../serialization/ByteArrayDeserializer.java | 23 +++++++------ .../serialization/ByteArraySerializer.java | 23 +++++++------ .../serialization/ByteBufferDeserializer.java | 22 ++++++++----- .../serialization/ByteBufferSerializer.java | 22 ++++++++----- .../serialization/BytesDeserializer.java | 22 ++++++++----- .../common/serialization/BytesSerializer.java | 22 ++++++++----- .../common/serialization/Deserializer.java | 23 +++++++------ .../serialization/DoubleDeserializer.java | 22 ++++++++----- .../serialization/DoubleSerializer.java | 22 ++++++++----- .../serialization/IntegerDeserializer.java | 22 ++++++++----- .../serialization/IntegerSerializer.java | 22 ++++++++----- .../serialization/LongDeserializer.java | 22 ++++++++----- .../common/serialization/LongSerializer.java | 22 ++++++++----- .../kafka/common/serialization/Serde.java | 23 +++++++------ .../kafka/common/serialization/Serdes.java | 23 +++++++------ .../common/serialization/Serializer.java | 23 +++++++------ .../serialization/StringDeserializer.java | 22 ++++++++----- .../serialization/StringSerializer.java | 22 ++++++++----- .../kafka/common/utils/AbstractIterator.java | 6 ++-- .../kafka/common/utils/AppInfoParser.java | 6 ++-- .../org/apache/kafka/common/utils/Bytes.java | 7 ++-- .../kafka/common/utils/CircularIterator.java | 9 +++-- .../kafka/common/utils/CollectionUtils.java | 22 ++++++++----- .../kafka/common/utils/CopyOnWriteMap.java | 26 ++++++++------- .../org/apache/kafka/common/utils/Crc32.java | 26 ++++++++------- .../org/apache/kafka/common/utils/Exit.java | 7 ++-- .../kafka/common/utils/KafkaThread.java | 26 ++++++++------- .../org/apache/kafka/common/utils/Shell.java | 19 +++++------ .../apache/kafka/common/utils/SystemTime.java | 6 ++-- .../org/apache/kafka/common/utils/Time.java | 6 ++-- .../org/apache/kafka/common/utils/Utils.java | 22 ++++++++----- .../server/policy/CreateTopicPolicy.java | 23 +++++++------ .../apache/kafka/clients/ClientUtilsTest.java | 6 ++-- .../apache/kafka/clients/MetadataTest.java | 22 ++++++++----- .../org/apache/kafka/clients/MockClient.java | 6 ++-- .../kafka/clients/NetworkClientTest.java | 6 ++-- .../kafka/clients/NodeApiVersionsTest.java | 6 ++-- .../clients/consumer/ConsumerConfigTest.java | 12 +++---- .../clients/consumer/ConsumerRecordTest.java | 14 ++++---- .../clients/consumer/ConsumerRecordsTest.java | 11 +++---- .../clients/consumer/KafkaConsumerTest.java | 6 ++-- .../clients/consumer/MockConsumerTest.java | 6 ++-- .../clients/consumer/RangeAssignorTest.java | 12 +++---- .../consumer/RoundRobinAssignorTest.java | 26 ++++++++------- ...izeCompatibilityOffsetAndMetadataTest.java | 23 +++++++------ .../internals/AbstractCoordinatorTest.java | 14 ++++---- .../internals/ConsumerCoordinatorTest.java | 6 ++-- .../internals/ConsumerInterceptorsTest.java | 6 ++-- .../internals/ConsumerNetworkClientTest.java | 22 ++++++++----- .../internals/ConsumerProtocolTest.java | 12 +++---- .../consumer/internals/FetcherTest.java | 6 ++-- .../consumer/internals/HeartbeatTest.java | 6 ++-- .../internals/MockPartitionAssignor.java | 12 +++---- .../consumer/internals/RequestFutureTest.java | 22 ++++++++----- .../internals/SubscriptionStateTest.java | 6 ++-- .../clients/producer/KafkaProducerTest.java | 8 ++--- .../clients/producer/MockProducerTest.java | 6 ++-- .../clients/producer/ProducerRecordTest.java | 6 ++-- .../clients/producer/RecordSendTest.java | 6 ++-- .../producer/internals/BufferPoolTest.java | 6 ++-- .../internals/DefaultPartitionerTest.java | 22 ++++++++----- .../internals/ProducerInterceptorsTest.java | 22 ++++++++----- .../internals/RecordAccumulatorTest.java | 26 ++++++++------- .../producer/internals/SenderTest.java | 22 ++++++++----- .../org/apache/kafka/common/ClusterTest.java | 22 ++++++++----- .../kafka/common/PartitionInfoTest.java | 22 ++++++++----- ...ializeCompatibilityTopicPartitionTest.java | 23 +++++++------ .../kafka/common/cache/LRUCacheTest.java | 9 +++-- .../common/config/AbstractConfigTest.java | 26 ++++++++------- .../kafka/common/config/ConfigDefTest.java | 22 ++++++++----- .../common/internals/PartitionStatesTest.java | 7 ++-- .../common/metrics/FakeMetricsReporter.java | 26 ++++++++------- .../kafka/common/metrics/JmxReporterTest.java | 6 ++-- .../kafka/common/metrics/MetricsTest.java | 26 ++++++++------- .../kafka/common/metrics/SensorTest.java | 22 ++++++++----- .../common/metrics/stats/HistogramTest.java | 6 ++-- .../kafka/common/network/CertStores.java | 22 ++++++++----- .../kafka/common/network/EchoServer.java | 22 ++++++++----- .../common/network/NetworkTestUtils.java | 22 ++++++++----- .../kafka/common/network/NioEchoServer.java | 22 ++++++++----- .../kafka/common/network/SelectorTest.java | 22 ++++++++----- .../kafka/common/network/SslSelectorTest.java | 22 ++++++++----- .../common/network/SslTransportLayerTest.java | 22 ++++++++----- .../kafka/common/protocol/ApiKeysTest.java | 7 ++-- .../kafka/common/protocol/ErrorsTest.java | 6 ++-- .../types/ProtocolSerializationTest.java | 6 ++-- .../record/ByteBufferLogInputStreamTest.java | 14 ++++---- .../common/record/CompressionTypeTest.java | 14 ++++---- .../kafka/common/record/FileRecordsTest.java | 14 ++++---- .../kafka/common/record/KafkaLZ4Test.java | 6 ++-- .../record/MemoryRecordsBuilderTest.java | 14 ++++---- .../common/record/MemoryRecordsTest.java | 6 ++-- .../kafka/common/record/RecordTest.java | 6 ++-- .../kafka/common/record/SimpleRecordTest.java | 6 ++-- .../common/record/TimestampTypeTest.java | 14 ++++---- .../common/requests/RequestResponseTest.java | 23 +++++++------ .../common/security/JaasContextTest.java | 12 +++---- .../common/security/TestSecurityConfig.java | 22 ++++++++----- .../security/auth/KafkaPrincipalTest.java | 12 +++---- .../authenticator/SaslAuthenticatorTest.java | 22 ++++++++----- .../authenticator/TestDigestLoginModule.java | 22 ++++++++----- .../authenticator/TestJaasConfig.java | 22 ++++++++----- .../security/kerberos/KerberosNameTest.java | 7 ++-- .../scram/ScramCredentialUtilsTest.java | 7 ++-- .../security/scram/ScramFormatterTest.java | 7 ++-- .../security/scram/ScramMessagesTest.java | 7 ++-- .../common/security/ssl/SslFactoryTest.java | 22 ++++++++----- .../serialization/SerializationTest.java | 22 ++++++++----- .../common/utils/AbstractIteratorTest.java | 6 ++-- .../apache/kafka/common/utils/CrcTest.java | 6 ++-- .../apache/kafka/common/utils/MockTime.java | 22 ++++++++----- .../apache/kafka/common/utils/Serializer.java | 23 +++++++------ .../apache/kafka/common/utils/UtilsTest.java | 6 ++-- .../org/apache/kafka/test/DelayedReceive.java | 6 ++-- .../org/apache/kafka/test/MetricsBench.java | 26 ++++++++------- .../apache/kafka/test/Microbenchmarks.java | 26 ++++++++------- .../test/MockClusterResourceListener.java | 7 ++-- .../kafka/test/MockConsumerInterceptor.java | 6 ++-- .../apache/kafka/test/MockDeserializer.java | 6 ++-- .../kafka/test/MockMetricsReporter.java | 6 ++-- .../apache/kafka/test/MockPartitioner.java | 6 ++-- .../kafka/test/MockProducerInterceptor.java | 6 ++-- .../org/apache/kafka/test/MockSelector.java | 22 ++++++++----- .../org/apache/kafka/test/MockSerializer.java | 6 ++-- .../org/apache/kafka/test/TestCondition.java | 30 ++++++++--------- .../org/apache/kafka/test/TestSslUtils.java | 7 ++-- .../java/org/apache/kafka/test/TestUtils.java | 12 +++---- .../connect/connector/ConnectRecord.java | 15 ++++----- .../kafka/connect/connector/Connector.java | 9 +++-- .../connect/connector/ConnectorContext.java | 9 +++-- .../apache/kafka/connect/connector/Task.java | 9 +++-- .../kafka/connect/data/ConnectSchema.java | 9 +++-- .../org/apache/kafka/connect/data/Date.java | 9 +++-- .../apache/kafka/connect/data/Decimal.java | 9 +++-- .../org/apache/kafka/connect/data/Field.java | 9 +++-- .../org/apache/kafka/connect/data/Schema.java | 9 +++-- .../kafka/connect/data/SchemaAndValue.java | 9 +++-- .../kafka/connect/data/SchemaBuilder.java | 9 +++-- .../kafka/connect/data/SchemaProjector.java | 27 +++++++-------- .../org/apache/kafka/connect/data/Struct.java | 9 +++-- .../org/apache/kafka/connect/data/Time.java | 9 +++-- .../apache/kafka/connect/data/Timestamp.java | 9 +++-- .../errors/AlreadyExistsException.java | 9 +++-- .../connect/errors/ConnectException.java | 9 +++-- .../kafka/connect/errors/DataException.java | 9 +++-- .../errors/IllegalWorkerStateException.java | 9 +++-- .../connect/errors/NotFoundException.java | 9 +++-- .../connect/errors/RetriableException.java | 9 +++-- .../errors/SchemaBuilderException.java | 9 +++-- .../errors/SchemaProjectorException.java | 26 ++++++++------- .../kafka/connect/sink/SinkConnector.java | 9 +++-- .../apache/kafka/connect/sink/SinkRecord.java | 15 ++++----- .../apache/kafka/connect/sink/SinkTask.java | 8 ++--- .../kafka/connect/sink/SinkTaskContext.java | 9 +++-- .../kafka/connect/source/SourceConnector.java | 8 ++--- .../kafka/connect/source/SourceRecord.java | 15 ++++----- .../kafka/connect/source/SourceTask.java | 9 +++-- .../connect/source/SourceTaskContext.java | 8 ++--- .../kafka/connect/storage/Converter.java | 9 +++-- .../connect/storage/OffsetStorageReader.java | 9 +++-- .../connect/storage/StringConverter.java | 9 +++-- .../connect/transforms/Transformation.java | 9 +++-- .../kafka/connect/util/ConnectorUtils.java | 9 +++-- .../ConnectorReconfigurationTest.java | 9 +++-- .../kafka/connect/data/ConnectSchemaTest.java | 9 +++-- .../apache/kafka/connect/data/DateTest.java | 9 +++-- .../kafka/connect/data/DecimalTest.java | 9 +++-- .../apache/kafka/connect/data/FakeSchema.java | 9 +++-- .../apache/kafka/connect/data/FieldTest.java | 9 +++-- .../kafka/connect/data/SchemaBuilderTest.java | 9 +++-- .../connect/data/SchemaProjectorTest.java | 27 +++++++-------- .../apache/kafka/connect/data/StructTest.java | 9 +++-- .../apache/kafka/connect/data/TimeTest.java | 9 +++-- .../kafka/connect/data/TimestampTest.java | 9 +++-- .../connect/storage/StringConverterTest.java | 9 +++-- .../connect/util/ConnectorUtilsTest.java | 9 +++-- .../connect/file/FileStreamSinkConnector.java | 9 +++-- .../connect/file/FileStreamSinkTask.java | 9 +++-- .../file/FileStreamSourceConnector.java | 9 +++-- .../connect/file/FileStreamSourceTask.java | 9 +++-- .../file/FileStreamSinkConnectorTest.java | 9 +++-- .../connect/file/FileStreamSinkTaskTest.java | 9 +++-- .../file/FileStreamSourceConnectorTest.java | 9 +++-- .../file/FileStreamSourceTaskTest.java | 9 +++-- .../kafka/connect/json/JsonConverter.java | 15 ++++----- .../kafka/connect/json/JsonDeserializer.java | 14 ++++---- .../apache/kafka/connect/json/JsonSchema.java | 9 +++-- .../kafka/connect/json/JsonSerializer.java | 14 ++++---- .../kafka/connect/json/JsonConverterTest.java | 9 +++-- .../kafka/connect/cli/ConnectDistributed.java | 9 +++-- .../kafka/connect/cli/ConnectStandalone.java | 15 ++++----- .../kafka/connect/runtime/AbstractHerder.java | 14 ++++---- .../kafka/connect/runtime/AbstractStatus.java | 14 ++++---- .../apache/kafka/connect/runtime/Connect.java | 15 ++++----- .../connect/runtime/ConnectorConfig.java | 15 ++++----- .../connect/runtime/ConnectorFactory.java | 29 ++++++++-------- .../connect/runtime/ConnectorStatus.java | 8 ++--- .../apache/kafka/connect/runtime/Herder.java | 15 ++++----- .../runtime/HerderConnectorContext.java | 15 ++++----- .../connect/runtime/PluginDiscovery.java | 15 ++++----- .../connect/runtime/SinkConnectorConfig.java | 15 ++++----- .../runtime/SourceConnectorConfig.java | 15 ++++----- .../runtime/SourceTaskOffsetCommitter.java | 15 ++++----- .../kafka/connect/runtime/TargetState.java | 14 ++++---- .../kafka/connect/runtime/TaskConfig.java | 9 +++-- .../kafka/connect/runtime/TaskStatus.java | 14 ++++---- .../connect/runtime/TransformationChain.java | 15 ++++----- .../apache/kafka/connect/runtime/Worker.java | 15 ++++----- .../kafka/connect/runtime/WorkerConfig.java | 15 ++++----- .../connect/runtime/WorkerConnector.java | 14 ++++---- .../kafka/connect/runtime/WorkerSinkTask.java | 15 ++++----- .../runtime/WorkerSinkTaskContext.java | 27 ++++++++------- .../connect/runtime/WorkerSourceTask.java | 15 ++++----- .../runtime/WorkerSourceTaskContext.java | 9 +++-- .../kafka/connect/runtime/WorkerTask.java | 15 ++++----- .../distributed/ClusterConfigState.java | 9 +++-- .../runtime/distributed/ConnectProtocol.java | 9 +++-- .../distributed/DistributedConfig.java | 9 +++-- .../distributed/DistributedHerder.java | 9 +++-- .../distributed/NotAssignedException.java | 14 ++++---- .../distributed/NotLeaderException.java | 9 +++-- .../distributed/RebalanceNeededException.java | 14 ++++---- .../distributed/RequestTargetException.java | 14 ++++---- .../distributed/WorkerCoordinator.java | 8 ++--- .../distributed/WorkerGroupMember.java | 9 +++-- .../distributed/WorkerRebalanceListener.java | 9 +++-- .../connect/runtime/rest/RestServer.java | 9 +++-- .../runtime/rest/entities/ConfigInfo.java | 15 ++++----- .../runtime/rest/entities/ConfigInfos.java | 15 ++++----- .../runtime/rest/entities/ConfigKeyInfo.java | 15 ++++----- .../rest/entities/ConfigValueInfo.java | 15 ++++----- .../runtime/rest/entities/ConnectorInfo.java | 9 +++-- .../rest/entities/ConnectorPluginInfo.java | 27 ++++++++------- .../rest/entities/ConnectorStateInfo.java | 14 ++++---- .../rest/entities/CreateConnectorRequest.java | 9 +++-- .../runtime/rest/entities/ErrorMessage.java | 9 +++-- .../runtime/rest/entities/ServerInfo.java | 9 +++-- .../runtime/rest/entities/TaskInfo.java | 9 +++-- .../rest/errors/BadRequestException.java | 14 ++++---- .../rest/errors/ConnectExceptionMapper.java | 9 +++-- .../rest/errors/ConnectRestException.java | 9 +++-- .../resources/ConnectorPluginsResource.java | 9 +++-- .../rest/resources/ConnectorsResource.java | 9 +++-- .../runtime/rest/resources/RootResource.java | 9 +++-- .../runtime/standalone/StandaloneConfig.java | 9 +++-- .../runtime/standalone/StandaloneHerder.java | 15 ++++----- .../connect/storage/ConfigBackingStore.java | 14 ++++---- .../storage/FileOffsetBackingStore.java | 15 ++++----- .../storage/KafkaConfigBackingStore.java | 15 ++++----- .../storage/KafkaOffsetBackingStore.java | 9 +++-- .../storage/KafkaStatusBackingStore.java | 14 ++++---- .../storage/MemoryConfigBackingStore.java | 14 ++++---- .../storage/MemoryOffsetBackingStore.java | 15 ++++----- .../storage/MemoryStatusBackingStore.java | 14 ++++---- .../connect/storage/OffsetBackingStore.java | 15 ++++----- .../storage/OffsetStorageReaderImpl.java | 15 ++++----- .../connect/storage/OffsetStorageWriter.java | 15 ++++----- .../kafka/connect/storage/OffsetUtils.java | 9 +++-- .../connect/storage/StatusBackingStore.java | 14 ++++---- .../kafka/connect/tools/MockConnector.java | 14 ++++---- .../connect/tools/MockSinkConnector.java | 14 ++++---- .../kafka/connect/tools/MockSinkTask.java | 14 ++++---- .../connect/tools/MockSourceConnector.java | 14 ++++---- .../kafka/connect/tools/MockSourceTask.java | 14 ++++---- .../connect/tools/SchemaSourceConnector.java | 9 +++-- .../kafka/connect/tools/SchemaSourceTask.java | 9 +++-- .../connect/tools/TransformationDoc.java | 15 ++++----- .../tools/VerifiableSinkConnector.java | 9 +++-- .../connect/tools/VerifiableSinkTask.java | 9 +++-- .../tools/VerifiableSourceConnector.java | 9 +++-- .../connect/tools/VerifiableSourceTask.java | 9 +++-- .../apache/kafka/connect/util/Callback.java | 15 ++++----- .../kafka/connect/util/ConnectUtils.java | 14 ++++---- .../kafka/connect/util/ConnectorTaskId.java | 15 ++++----- .../util/ConvertingFutureCallback.java | 9 +++-- .../kafka/connect/util/FutureCallback.java | 15 ++++----- .../kafka/connect/util/KafkaBasedLog.java | 9 +++-- .../kafka/connect/util/ReflectionsUtil.java | 30 +++++++++-------- .../connect/util/ShutdownableThread.java | 15 ++++----- .../apache/kafka/connect/util/SinkUtils.java | 14 ++++---- .../org/apache/kafka/connect/util/Table.java | 14 ++++---- .../connect/runtime/AbstractHerderTest.java | 14 ++++---- .../connect/runtime/ConnectorConfigTest.java | 15 ++++----- .../SourceTaskOffsetCommitterTest.java | 15 ++++----- .../connect/runtime/WorkerConnectorTest.java | 14 ++++---- .../connect/runtime/WorkerSinkTaskTest.java | 9 +++-- .../runtime/WorkerSinkTaskThreadedTest.java | 15 ++++----- .../connect/runtime/WorkerSourceTaskTest.java | 15 ++++----- .../kafka/connect/runtime/WorkerTaskTest.java | 14 ++++---- .../kafka/connect/runtime/WorkerTest.java | 15 ++++----- .../distributed/DistributedHerderTest.java | 9 +++-- .../distributed/WorkerCoordinatorTest.java | 8 ++--- .../connect/runtime/rest/RestServerTest.java | 9 +++-- .../ConnectorPluginsResourceTest.java | 9 +++-- .../resources/ConnectorsResourceTest.java | 9 +++-- .../standalone/StandaloneHerderTest.java | 15 ++++----- .../storage/FileOffsetBackingStoreTest.java | 15 ++++----- .../storage/KafkaConfigBackingStoreTest.java | 9 +++-- .../storage/KafkaOffsetBackingStoreTest.java | 9 +++-- .../storage/KafkaStatusBackingStoreTest.java | 14 ++++---- .../storage/MemoryStatusBackingStoreTest.java | 14 ++++---- .../storage/OffsetStorageWriterTest.java | 15 ++++----- .../util/ByteArrayProducerRecordEquals.java | 9 +++-- .../kafka/connect/util/KafkaBasedLogTest.java | 9 +++-- .../apache/kafka/connect/util/MockTime.java | 8 ++--- .../connect/util/ShutdownableThreadTest.java | 15 ++++----- .../apache/kafka/connect/util/TableTest.java | 14 ++++---- .../TestBackgroundThreadExceptionHandler.java | 15 ++++----- .../apache/kafka/connect/util/TestFuture.java | 9 +++-- .../kafka/connect/util/ThreadedTest.java | 15 ++++----- .../connect/transforms/ExtractField.java | 15 ++++----- .../kafka/connect/transforms/HoistField.java | 15 ++++----- .../kafka/connect/transforms/InsertField.java | 15 ++++----- .../kafka/connect/transforms/MaskField.java | 15 ++++----- .../kafka/connect/transforms/RegexRouter.java | 15 ++++----- .../connect/transforms/ReplaceField.java | 15 ++++----- .../connect/transforms/SetSchemaMetadata.java | 15 ++++----- .../connect/transforms/TimestampRouter.java | 15 ++++----- .../kafka/connect/transforms/ValueToKey.java | 15 ++++----- .../util/NonEmptyListValidator.java | 15 ++++----- .../transforms/util/RegexValidator.java | 15 ++++----- .../connect/transforms/util/Requirements.java | 15 ++++----- .../connect/transforms/util/SchemaUtil.java | 15 ++++----- .../connect/transforms/util/SimpleConfig.java | 15 ++++----- .../connect/transforms/ExtractFieldTest.java | 15 ++++----- .../connect/transforms/HoistFieldTest.java | 15 ++++----- .../connect/transforms/InsertFieldTest.java | 15 ++++----- .../connect/transforms/MaskFieldTest.java | 15 ++++----- .../connect/transforms/RegexRouterTest.java | 15 ++++----- .../connect/transforms/ReplaceFieldTest.java | 15 ++++----- .../transforms/SetSchemaMetadataTest.java | 15 ++++----- .../transforms/TimestampRouterTest.java | 15 ++++----- .../connect/transforms/ValueToKeyTest.java | 15 ++++----- .../kafka/consumer/ConsumerConnector.scala | 9 +++-- .../javaapi/consumer/ConsumerConnector.java | 9 +++-- .../consumer/ConsumerRebalanceListener.java | 7 ++-- .../main/scala/kafka/metrics/KafkaTimer.scala | 7 ++-- .../scala/kafka/server/LeaderElector.scala | 6 ++-- .../scala/kafka/tools/StreamsResetter.java | 12 +++---- .../main/java/kafka/examples/Consumer.java | 6 ++-- .../examples/KafkaConsumerProducerDemo.java | 6 ++-- .../java/kafka/examples/KafkaProperties.java | 12 +++---- .../main/java/kafka/examples/Producer.java | 6 ++-- .../kafka/examples/SimpleConsumerDemo.java | 6 ++-- .../log4jappender/KafkaLog4jAppender.java | 7 ++-- .../log4jappender/KafkaLog4jAppenderTest.java | 6 ++-- .../log4jappender/MockKafkaLog4jAppender.java | 6 ++-- .../pageview/JsonPOJODeserializer.java | 14 ++++---- .../examples/pageview/JsonPOJOSerializer.java | 14 ++++---- .../pageview/JsonTimestampExtractor.java | 7 ++-- .../examples/pageview/PageViewTypedDemo.java | 7 ++-- .../pageview/PageViewUntypedDemo.java | 7 ++-- .../kafka/streams/examples/pipe/PipeDemo.java | 7 ++-- .../examples/wordcount/WordCountDemo.java | 7 ++-- .../wordcount/WordCountProcessorDemo.java | 7 ++-- .../kafka/streams/KafkaClientSupplier.java | 13 ++++---- .../apache/kafka/streams/KafkaStreams.java | 13 ++++---- .../org/apache/kafka/streams/KeyValue.java | 7 ++-- .../apache/kafka/streams/StreamsConfig.java | 13 ++++---- .../apache/kafka/streams/StreamsMetrics.java | 7 ++-- .../errors/InvalidStateStoreException.java | 24 +++++++------- .../kafka/streams/errors/LockException.java | 24 +++++++------- .../errors/ProcessorStateException.java | 7 ++-- .../streams/errors/StreamsException.java | 9 +++-- .../errors/TaskAssignmentException.java | 6 ++-- .../streams/errors/TaskIdFormatException.java | 7 ++-- .../errors/TopologyBuilderException.java | 7 ++-- .../kafka/streams/kstream/Aggregator.java | 13 ++++---- .../kafka/streams/kstream/ForeachAction.java | 12 +++---- .../kafka/streams/kstream/GlobalKTable.java | 13 ++++---- .../kafka/streams/kstream/Initializer.java | 7 ++-- .../kafka/streams/kstream/JoinWindows.java | 13 ++++---- .../kafka/streams/kstream/KGroupedStream.java | 28 ++++++++-------- .../kafka/streams/kstream/KGroupedTable.java | 7 ++-- .../apache/kafka/streams/kstream/KStream.java | 12 +++---- .../kafka/streams/kstream/KStreamBuilder.java | 7 ++-- .../apache/kafka/streams/kstream/KTable.java | 7 ++-- .../kafka/streams/kstream/KeyValueMapper.java | 12 +++---- .../apache/kafka/streams/kstream/Merger.java | 7 ++-- .../kafka/streams/kstream/Predicate.java | 6 ++-- .../apache/kafka/streams/kstream/Reducer.java | 13 ++++---- .../kafka/streams/kstream/SessionWindows.java | 12 +++---- .../kafka/streams/kstream/TimeWindows.java | 13 ++++---- .../kafka/streams/kstream/Transformer.java | 7 ++-- .../streams/kstream/TransformerSupplier.java | 7 ++-- .../streams/kstream/UnlimitedWindows.java | 13 ++++---- .../kafka/streams/kstream/ValueJoiner.java | 6 ++-- .../kafka/streams/kstream/ValueMapper.java | 6 ++-- .../streams/kstream/ValueTransformer.java | 7 ++-- .../kstream/ValueTransformerSupplier.java | 6 ++-- .../apache/kafka/streams/kstream/Window.java | 12 +++---- .../kafka/streams/kstream/Windowed.java | 7 ++-- .../apache/kafka/streams/kstream/Windows.java | 12 +++---- ...ctKTableKTableJoinValueGetterSupplier.java | 12 +++---- .../kstream/internals/AbstractStream.java | 7 ++-- .../kstream/internals/CacheFlushListener.java | 12 +++---- .../streams/kstream/internals/Change.java | 7 ++-- .../internals/ChangedDeserializer.java | 7 ++-- .../kstream/internals/ChangedSerializer.java | 7 ++-- .../ForwardingCacheFlushListener.java | 12 +++---- .../kstream/internals/GlobalKTableImpl.java | 12 +++---- .../kstream/internals/KGroupedStreamImpl.java | 28 ++++++++-------- .../kstream/internals/KGroupedTableImpl.java | 7 ++-- .../KStreamAggProcessorSupplier.java | 7 ++-- .../kstream/internals/KStreamAggregate.java | 7 ++-- .../kstream/internals/KStreamBranch.java | 7 ++-- .../kstream/internals/KStreamFilter.java | 7 ++-- .../kstream/internals/KStreamFlatMap.java | 7 ++-- .../internals/KStreamFlatMapValues.java | 7 ++-- .../kstream/internals/KStreamForeach.java | 7 ++-- .../internals/KStreamGlobalKTableJoin.java | 13 ++++---- .../kstream/internals/KStreamImpl.java | 13 ++++---- .../kstream/internals/KStreamJoinWindow.java | 7 ++-- .../kstream/internals/KStreamKStreamJoin.java | 13 ++++---- .../kstream/internals/KStreamKTableJoin.java | 13 ++++---- .../internals/KStreamKTableJoinProcessor.java | 12 +++---- .../streams/kstream/internals/KStreamMap.java | 7 ++-- .../kstream/internals/KStreamMapValues.java | 7 ++-- .../kstream/internals/KStreamPassThrough.java | 7 ++-- .../kstream/internals/KStreamPeek.java | 7 ++-- .../kstream/internals/KStreamReduce.java | 7 ++-- .../KStreamSessionWindowAggregate.java | 13 ++++---- .../kstream/internals/KStreamTransform.java | 7 ++-- .../internals/KStreamTransformValues.java | 7 ++-- .../internals/KStreamWindowAggregate.java | 7 ++-- .../internals/KStreamWindowReduce.java | 7 ++-- .../kstream/internals/KTableAggregate.java | 7 ++-- .../kstream/internals/KTableFilter.java | 7 ++-- .../streams/kstream/internals/KTableImpl.java | 13 ++++---- .../internals/KTableKTableAbstractJoin.java | 7 ++-- .../kstream/internals/KTableKTableJoin.java | 13 ++++---- .../internals/KTableKTableJoinMerger.java | 7 ++-- .../KTableKTableJoinValueGetter.java | 12 +++---- .../internals/KTableKTableLeftJoin.java | 13 ++++---- .../KTableKTableLeftJoinValueGetter.java | 12 +++---- .../internals/KTableKTableOuterJoin.java | 13 ++++---- .../internals/KTableKTableRightJoin.java | 13 ++++---- .../kstream/internals/KTableMapValues.java | 7 ++-- .../internals/KTableProcessorSupplier.java | 7 ++-- .../kstream/internals/KTableReduce.java | 7 ++-- .../internals/KTableRepartitionMap.java | 7 ++-- .../kstream/internals/KTableSource.java | 7 ++-- .../KTableSourceValueGetterSupplier.java | 7 ++-- .../kstream/internals/KTableValueGetter.java | 7 ++-- .../internals/KTableValueGetterSupplier.java | 7 ++-- .../kstream/internals/KeyValuePrinter.java | 13 ++++---- .../kstream/internals/SessionKeySerde.java | 12 +++---- .../kstream/internals/SessionWindow.java | 32 +++++++++--------- .../streams/kstream/internals/TimeWindow.java | 6 ++-- .../kstream/internals/TupleForwarder.java | 7 ++-- .../kstream/internals/UnlimitedWindow.java | 12 +++---- .../internals/WindowedDeserializer.java | 7 ++-- .../kstream/internals/WindowedSerializer.java | 7 ++-- .../internals/WindowedStreamPartitioner.java | 6 ++-- .../streams/processor/AbstractProcessor.java | 4 +-- .../processor/DefaultPartitionGrouper.java | 7 ++-- .../ExtractRecordMetadataTimestamp.java | 7 ++-- .../processor/FailOnInvalidTimestamp.java | 7 ++-- .../LogAndSkipOnInvalidTimestamp.java | 7 ++-- .../streams/processor/PartitionGrouper.java | 7 ++-- .../kafka/streams/processor/Processor.java | 7 ++-- .../streams/processor/ProcessorContext.java | 7 ++-- .../streams/processor/ProcessorSupplier.java | 7 ++-- .../processor/StateRestoreCallback.java | 7 ++-- .../kafka/streams/processor/StateStore.java | 13 ++++---- .../streams/processor/StateStoreSupplier.java | 13 ++++---- .../streams/processor/StreamPartitioner.java | 6 ++-- .../kafka/streams/processor/TaskId.java | 7 ++-- .../streams/processor/TimestampExtractor.java | 7 ++-- .../streams/processor/TopologyBuilder.java | 13 ++++---- .../UsePreviousTimeOnInvalidTimestamp.java | 7 ++-- .../WallclockTimestampExtractor.java | 7 ++-- .../internals/AbstractProcessorContext.java | 12 +++---- .../processor/internals/AbstractTask.java | 7 ++-- .../processor/internals/ChangelogReader.java | 12 +++---- .../processor/internals/Checkpointable.java | 12 +++---- .../internals/DefaultKafkaClientSupplier.java | 13 ++++---- .../internals/DefaultStreamPartitioner.java | 12 +++---- .../internals/GlobalProcessorContextImpl.java | 7 ++-- .../internals/GlobalStateMaintainer.java | 12 +++---- .../internals/GlobalStateManager.java | 12 +++---- .../internals/GlobalStateManagerImpl.java | 12 +++---- .../internals/GlobalStateUpdateTask.java | 12 +++---- .../internals/GlobalStreamThread.java | 12 +++---- .../internals/InternalProcessorContext.java | 12 +++---- .../internals/InternalTopicConfig.java | 12 +++---- .../internals/InternalTopicManager.java | 7 ++-- .../internals/MinTimestampTracker.java | 7 ++-- .../processor/internals/PartitionGroup.java | 13 ++++---- .../internals/ProcessorContextImpl.java | 7 ++-- .../processor/internals/ProcessorNode.java | 13 ++++---- .../internals/ProcessorRecordContext.java | 12 +++---- .../internals/ProcessorStateManager.java | 13 ++++---- .../internals/ProcessorTopology.java | 7 ++-- .../processor/internals/PunctuationQueue.java | 7 ++-- .../internals/PunctuationSchedule.java | 7 ++-- .../processor/internals/Punctuator.java | 7 ++-- .../processor/internals/QuickUnion.java | 7 ++-- .../processor/internals/RecordCollector.java | 12 +++---- .../internals/RecordCollectorImpl.java | 7 ++-- .../processor/internals/RecordContext.java | 12 +++---- .../internals/RecordDeserializer.java | 12 +++---- .../processor/internals/RecordQueue.java | 7 ++-- .../streams/processor/internals/SinkNode.java | 7 ++-- .../processor/internals/SourceNode.java | 7 ++-- .../SourceNodeRecordDeserializer.java | 12 +++---- .../streams/processor/internals/Stamped.java | 7 ++-- .../processor/internals/StampedRecord.java | 7 ++-- .../internals/StandbyContextImpl.java | 7 ++-- .../processor/internals/StandbyTask.java | 13 ++++---- .../processor/internals/StateDirectory.java | 12 +++---- .../processor/internals/StateManager.java | 12 +++---- .../processor/internals/StateRestorer.java | 12 +++---- .../internals/StoreChangelogReader.java | 12 +++---- .../internals/StreamPartitionAssignor.java | 7 ++-- .../processor/internals/StreamTask.java | 13 ++++---- .../processor/internals/StreamThread.java | 13 ++++---- .../internals/StreamsKafkaClient.java | 23 +++++++------ .../internals/StreamsMetadataState.java | 12 +++---- .../internals/StreamsMetricsImpl.java | 13 ++++---- .../processor/internals/TimestampTracker.java | 7 ++-- .../internals/assignment/AssignmentInfo.java | 13 ++++---- .../internals/assignment/ClientState.java | 7 ++-- .../assignment/StickyTaskAssignor.java | 12 +++---- .../assignment/SubscriptionInfo.java | 13 ++++---- .../internals/assignment/TaskAssignor.java | 12 +++---- .../apache/kafka/streams/state/HostInfo.java | 12 +++---- .../kafka/streams/state/KeyValueIterator.java | 33 +++++++++---------- .../kafka/streams/state/KeyValueStore.java | 33 +++++++++---------- .../streams/state/QueryableStoreType.java | 24 +++++++------- .../streams/state/QueryableStoreTypes.java | 24 +++++++------- .../streams/state/ReadOnlyKeyValueStore.java | 24 +++++++------- .../streams/state/ReadOnlySessionStore.java | 12 +++---- .../streams/state/ReadOnlyWindowStore.java | 24 +++++++------- .../streams/state/RocksDBConfigSetter.java | 12 +++---- .../kafka/streams/state/SessionStore.java | 12 +++---- .../kafka/streams/state/StateSerdes.java | 4 +-- .../apache/kafka/streams/state/Stores.java | 4 +-- .../kafka/streams/state/StreamsMetadata.java | 12 +++---- .../kafka/streams/state/WindowStore.java | 33 +++++++++---------- .../streams/state/WindowStoreIterator.java | 33 +++++++++---------- ...bstractMergedSortedCacheStoreIterator.java | 33 +++++++++---------- .../internals/AbstractStoreSupplier.java | 12 +++---- .../state/internals/CachedStateStore.java | 12 +++---- .../state/internals/CachingKeyValueStore.java | 12 +++---- .../state/internals/CachingSessionStore.java | 12 +++---- .../state/internals/CachingWindowStore.java | 12 +++---- .../ChangeLoggingKeyValueBytesStore.java | 12 +++---- .../internals/ChangeLoggingKeyValueStore.java | 12 +++---- .../ChangeLoggingSegmentedBytesStore.java | 12 +++---- .../CompositeReadOnlyKeyValueStore.java | 28 ++++++++-------- .../CompositeReadOnlySessionStore.java | 12 +++---- .../CompositeReadOnlyWindowStore.java | 24 +++++++------- .../DelegatingPeekingKeyValueIterator.java | 12 +++---- .../internals/GlobalStateStoreProvider.java | 12 +++---- .../state/internals/HasNextCondition.java | 12 +++---- .../InMemoryKeyValueLoggedStore.java | 7 ++-- .../internals/InMemoryKeyValueStore.java | 24 +++++++------- .../InMemoryKeyValueStoreSupplier.java | 7 ++-- .../InMemoryLRUCacheStoreSupplier.java | 7 ++-- .../state/internals/LRUCacheEntry.java | 8 ++--- .../state/internals/MemoryLRUCache.java | 4 +-- .../internals/MemoryNavigableLRUCache.java | 4 +-- ...ergedSortedCacheKeyValueStoreIterator.java | 12 +++---- ...MergedSortedCacheSessionStoreIterator.java | 32 +++++++++--------- .../MergedSortedCacheWindowStoreIterator.java | 12 +++---- .../state/internals/MeteredKeyValueStore.java | 7 ++-- .../internals/MeteredSegmentedBytesStore.java | 7 ++-- .../streams/state/internals/NamedCache.java | 12 +++---- .../state/internals/OffsetCheckpoint.java | 13 ++++---- .../internals/PeekingKeyValueIterator.java | 12 +++---- .../internals/QueryableStoreProvider.java | 28 ++++++++-------- .../RocksDBKeyValueStoreSupplier.java | 7 ++-- .../internals/RocksDBSegmentedBytesStore.java | 12 +++---- .../state/internals/RocksDBSessionStore.java | 12 +++---- .../RocksDBSessionStoreSupplier.java | 13 ++++---- .../streams/state/internals/RocksDBStore.java | 7 ++-- .../state/internals/RocksDBWindowStore.java | 33 +++++++++---------- .../internals/RocksDBWindowStoreSupplier.java | 7 ++-- .../streams/state/internals/Segment.java | 12 +++---- .../state/internals/SegmentIterator.java | 12 +++---- .../state/internals/SegmentedBytesStore.java | 12 +++---- .../streams/state/internals/Segments.java | 12 +++---- .../internals/SerializedKeyValueIterator.java | 12 +++---- .../state/internals/SessionKeySchema.java | 12 +++---- .../state/internals/StateStoreProvider.java | 24 +++++++------- .../state/internals/StoreChangeLogger.java | 7 ++-- .../StreamThreadStateStoreProvider.java | 24 +++++++------- .../streams/state/internals/ThreadCache.java | 8 ++--- .../state/internals/WindowKeySchema.java | 12 +++---- .../state/internals/WindowStoreSupplier.java | 12 +++---- .../state/internals/WindowStoreUtils.java | 33 +++++++++---------- .../WrappedSessionStoreIterator.java | 32 +++++++++--------- .../state/internals/WrappedStateStore.java | 32 +++++++++--------- .../internals/WrappedWindowStoreIterator.java | 32 +++++++++--------- .../internals/WrappingStoreProvider.java | 24 +++++++------- .../kafka/streams/KafkaStreamsTest.java | 13 ++++---- .../apache/kafka/streams/KeyValueTest.java | 13 ++++---- .../kafka/streams/StreamsConfigTest.java | 13 ++++---- .../integration/FanoutIntegrationTest.java | 13 ++++---- .../GlobalKTableIntegrationTest.java | 28 ++++++++-------- .../InternalTopicIntegrationTest.java | 12 +++---- .../integration/JoinIntegrationTest.java | 12 +++---- ...StreamAggregationDedupIntegrationTest.java | 24 +++++++++----- .../KStreamAggregationIntegrationTest.java | 28 ++++++++-------- .../KStreamKTableJoinIntegrationTest.java | 28 ++++++++-------- .../KStreamRepartitionJoinTest.java | 28 ++++++++-------- ...msFineGrainedAutoResetIntegrationTest.java | 30 ++++++++--------- .../KTableKTableJoinIntegrationTest.java | 28 ++++++++-------- .../QueryableStateIntegrationTest.java | 28 ++++++++-------- .../RegexSourceIntegrationTest.java | 30 ++++++++--------- .../integration/ResetIntegrationTest.java | 13 ++++---- .../utils/EmbeddedKafkaCluster.java | 13 ++++---- .../utils/IntegrationTestUtils.java | 13 ++++---- .../integration/utils/KafkaEmbedded.java | 13 ++++---- .../streams/kstream/JoinWindowsTest.java | 33 +++++++++---------- .../streams/kstream/KStreamBuilderTest.java | 7 ++-- .../streams/kstream/SessionWindowsTest.java | 33 +++++++++---------- .../streams/kstream/TimeWindowsTest.java | 33 +++++++++---------- .../streams/kstream/UnlimitedWindowsTest.java | 33 +++++++++---------- .../kafka/streams/kstream/WindowTest.java | 32 +++++++++--------- .../kafka/streams/kstream/WindowsTest.java | 12 +++---- .../internals/GlobalKTableJoinsTest.java | 12 +++---- .../internals/KGroupedStreamImplTest.java | 13 ++++---- .../internals/KGroupedTableImplTest.java | 7 ++-- .../kstream/internals/KStreamBranchTest.java | 7 ++-- .../kstream/internals/KStreamFilterTest.java | 7 ++-- .../kstream/internals/KStreamFlatMapTest.java | 7 ++-- .../internals/KStreamFlatMapValuesTest.java | 7 ++-- .../kstream/internals/KStreamForeachTest.java | 7 ++-- .../kstream/internals/KStreamImplTest.java | 7 ++-- .../internals/KStreamKStreamJoinTest.java | 7 ++-- .../internals/KStreamKStreamLeftJoinTest.java | 13 ++++---- .../internals/KStreamKTableJoinTest.java | 13 ++++---- .../internals/KStreamKTableLeftJoinTest.java | 7 ++-- .../kstream/internals/KStreamMapTest.java | 7 ++-- .../internals/KStreamMapValuesTest.java | 7 ++-- .../kstream/internals/KStreamPeekTest.java | 19 +++++------ .../internals/KStreamSelectKeyTest.java | 9 ++--- ...amSessionWindowAggregateProcessorTest.java | 13 ++++---- .../internals/KStreamTransformTest.java | 7 ++-- .../internals/KStreamTransformValuesTest.java | 7 ++-- .../internals/KStreamWindowAggregateTest.java | 13 ++++---- .../internals/KTableAggregateTest.java | 7 ++-- .../kstream/internals/KTableFilterTest.java | 7 ++-- .../kstream/internals/KTableForeachTest.java | 7 ++-- .../kstream/internals/KTableImplTest.java | 7 ++-- .../internals/KTableKTableJoinTest.java | 13 ++++---- .../internals/KTableKTableLeftJoinTest.java | 13 ++++---- .../internals/KTableKTableOuterJoinTest.java | 13 ++++---- .../kstream/internals/KTableMapKeysTest.java | 7 ++-- .../internals/KTableMapValuesTest.java | 7 ++-- .../kstream/internals/KTableSourceTest.java | 7 ++-- .../KeyValuePrinterProcessorTest.java | 13 ++++---- .../internals/SessionKeySerdeTest.java | 13 ++++---- .../kstream/internals/SessionWindowTest.java | 32 +++++++++--------- .../kstream/internals/TimeWindowTest.java | 32 +++++++++--------- .../internals/UnlimitedWindowTest.java | 32 +++++++++--------- .../WindowedStreamPartitionerTest.java | 7 ++-- .../kafka/streams/perf/SimpleBenchmark.java | 7 ++-- .../DefaultPartitionGrouperTest.java | 7 ++-- .../processor/FailOnInvalidTimestampTest.java | 12 +++---- .../LogAndSkipOnInvalidTimestampTest.java | 12 +++---- .../processor/TimestampExtractorTest.java | 12 +++---- .../processor/TopologyBuilderTest.java | 7 ++-- ...UsePreviousTimeOnInvalidTimestampTest.java | 12 +++---- .../WallclockTimestampExtractorTest.java | 12 +++---- .../AbstractProcessorContextTest.java | 13 ++++---- .../processor/internals/AbstractTaskTest.java | 13 ++++---- .../CopartitionedTopicsValidatorTest.java | 13 ++++---- .../internals/GlobalStateManagerImplTest.java | 13 ++++---- .../internals/GlobalStateTaskTest.java | 13 ++++---- .../internals/GlobalStreamThreadTest.java | 13 ++++---- .../internals/InternalTopicConfigTest.java | 13 ++++---- .../internals/InternalTopicManagerTest.java | 13 ++++---- .../internals/MinTimestampTrackerTest.java | 7 ++-- .../internals/MockStreamsMetrics.java | 12 +++---- .../internals/PartitionGroupTest.java | 13 ++++---- .../internals/ProcessorNodeTest.java | 13 ++++---- .../internals/ProcessorStateManagerTest.java | 7 ++-- .../internals/ProcessorTopologyTest.java | 7 ++-- .../internals/PunctuationQueueTest.java | 7 ++-- .../processor/internals/QuickUnionTest.java | 7 ++-- .../internals/RecordCollectorTest.java | 16 ++++----- .../internals/RecordContextStub.java | 12 +++---- .../processor/internals/RecordQueueTest.java | 13 ++++---- .../SingleGroupPartitionGrouperStub.java | 13 ++++---- .../processor/internals/SinkNodeTest.java | 13 ++++---- .../SourceNodeRecordDeserializerTest.java | 13 ++++---- .../processor/internals/StandbyTaskTest.java | 7 ++-- .../internals/StateConsumerTest.java | 13 ++++---- .../internals/StateDirectoryTest.java | 13 ++++---- .../processor/internals/StateManagerStub.java | 12 +++---- .../internals/StateRestorerTest.java | 13 ++++---- .../internals/StoreChangelogReaderTest.java | 13 ++++---- .../StreamPartitionAssignorTest.java | 7 ++-- .../processor/internals/StreamTaskTest.java | 7 ++-- .../processor/internals/StreamThreadTest.java | 13 ++++---- .../internals/StreamsKafkaClientTest.java | 29 ++++++++-------- .../internals/StreamsMetadataStateTest.java | 12 +++---- .../internals/StreamsMetricsImplTest.java | 12 +++---- .../assignment/AssignmentInfoTest.java | 7 ++-- .../internals/assignment/ClientStateTest.java | 13 ++++---- .../assignment/StickyTaskAssignorTest.java | 12 +++---- .../assignment/SubscriptionInfoTest.java | 7 ++-- .../state/KeyValueStoreTestDriver.java | 12 +++---- .../kafka/streams/state/NoOpWindowStore.java | 12 +++---- .../kafka/streams/state/StoresTest.java | 13 ++++---- .../internals/AbstractKeyValueStoreTest.java | 7 ++-- .../internals/CachingKeyValueStoreTest.java | 13 ++++---- .../internals/CachingSessionStoreTest.java | 13 ++++---- .../internals/CachingWindowStoreTest.java | 13 ++++---- .../ChangeLoggingKeyValueBytesStoreTest.java | 12 +++---- .../ChangeLoggingKeyValueStoreTest.java | 12 +++---- .../ChangeLoggingSegmentedBytesStoreTest.java | 13 ++++---- .../CompositeReadOnlyKeyValueStoreTest.java | 24 +++++++------- .../CompositeReadOnlySessionStoreTest.java | 13 ++++---- .../CompositeReadOnlyWindowStoreTest.java | 24 +++++++------- ...DelegatingPeekingKeyValueIteratorTest.java | 13 ++++---- .../GlobalStateStoreProviderTest.java | 13 ++++---- .../internals/InMemoryKeyValueStoreTest.java | 6 ++-- .../internals/InMemoryLRUCacheStoreTest.java | 4 +-- ...dSortedCacheKeyValueStoreIteratorTest.java | 13 ++++---- ...dCacheWrappedSessionStoreIteratorTest.java | 13 ++++---- ...edCacheWrappedWindowStoreIteratorTest.java | 13 ++++---- .../MeteredSegmentedBytesStoreTest.java | 13 ++++---- .../state/internals/NamedCacheTest.java | 13 ++++---- .../state/internals/OffsetCheckpointTest.java | 13 ++++---- .../internals/QueryableStoreProviderTest.java | 24 +++++++------- .../internals/ReadOnlyWindowStoreStub.java | 24 +++++++------- .../RocksDBKeyValueStoreSupplierTest.java | 13 ++++---- .../internals/RocksDBKeyValueStoreTest.java | 6 ++-- .../RocksDBSegmentedBytesStoreTest.java | 13 ++++---- .../RocksDBSessionStoreSupplierTest.java | 13 ++++---- .../internals/RocksDBSessionStoreTest.java | 13 ++++---- .../RocksDBWindowStoreSupplierTest.java | 13 ++++---- .../internals/RocksDBWindowStoreTest.java | 7 ++-- .../state/internals/SegmentIteratorTest.java | 13 ++++---- .../streams/state/internals/SegmentsTest.java | 13 ++++---- .../SerializedKeyValueIteratorTest.java | 13 ++++---- .../state/internals/StateStoreTestUtils.java | 12 +++---- .../internals/StoreChangeLoggerTest.java | 33 +++++++++---------- .../StreamThreadStateStoreProviderTest.java | 24 +++++++------- .../state/internals/ThreadCacheTest.java | 8 ++--- .../state/internals/WindowStoreUtilsTest.java | 32 +++++++++--------- .../internals/WrappingStoreProviderTest.java | 12 +++---- .../tests/BrokerCompatibilityTest.java | 6 ++-- .../streams/tests/ShutdownDeadlockTest.java | 12 +++---- .../kafka/streams/tests/SmokeTestClient.java | 7 ++-- .../kafka/streams/tests/SmokeTestDriver.java | 7 ++-- .../kafka/streams/tests/SmokeTestUtil.java | 7 ++-- .../kafka/streams/tests/StreamsSmokeTest.java | 7 ++-- .../kafka/test/GlobalStateManagerStub.java | 12 +++---- .../apache/kafka/test/KStreamTestDriver.java | 7 ++-- .../kafka/test/KTableValueGetterStub.java | 12 +++---- .../kafka/test/KeyValueIteratorStub.java | 12 +++---- .../org/apache/kafka/test/MockAggregator.java | 7 ++-- .../kafka/test/MockChangelogReader.java | 12 +++---- .../apache/kafka/test/MockClientSupplier.java | 7 ++-- .../apache/kafka/test/MockInitializer.java | 7 ++-- .../kafka/test/MockInternalTopicManager.java | 7 ++-- .../apache/kafka/test/MockKeyValueMapper.java | 7 ++-- .../kafka/test/MockProcessorContext.java | 7 ++-- .../apache/kafka/test/MockProcessorNode.java | 7 ++-- .../kafka/test/MockProcessorSupplier.java | 7 ++-- .../org/apache/kafka/test/MockReducer.java | 7 ++-- .../kafka/test/MockRestoreCallback.java | 12 +++---- .../kafka/test/MockRestoreConsumer.java | 12 +++---- .../org/apache/kafka/test/MockSourceNode.java | 7 ++-- .../kafka/test/MockStateStoreSupplier.java | 7 ++-- .../kafka/test/MockTimestampExtractor.java | 7 ++-- .../apache/kafka/test/MockValueJoiner.java | 7 ++-- .../kafka/test/NoOpProcessorContext.java | 12 +++---- .../apache/kafka/test/NoOpReadOnlyStore.java | 12 +++---- .../kafka/test/NoOpRecordCollector.java | 12 +++---- .../test/ProcessorTopologyTestDriver.java | 12 +++---- .../kafka/test/ReadOnlySessionStoreStub.java | 12 +++---- .../kafka/test/SegmentedBytesStoreStub.java | 12 +++---- .../kafka/test/StateStoreProviderStub.java | 24 +++++++------- .../apache/kafka/test/StreamsTestUtils.java | 28 ++++++++-------- .../kafka/tools/ClientCompatibilityTest.java | 12 +++---- .../kafka/tools/ProducerPerformance.java | 22 ++++++++----- .../kafka/tools/ThroughputThrottler.java | 7 ++-- .../kafka/tools/VerifiableConsumer.java | 7 ++-- .../kafka/tools/VerifiableLog4jAppender.java | 7 ++-- .../kafka/tools/VerifiableProducer.java | 7 ++-- 1100 files changed, 7765 insertions(+), 7406 deletions(-) create mode 100644 checkstyle/java.header diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 25d6f2fd07851..51b613dfe0197 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -24,8 +24,8 @@ - - + + diff --git a/checkstyle/java.header b/checkstyle/java.header new file mode 100644 index 0000000000000..45fd2d5c2daf7 --- /dev/null +++ b/checkstyle/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. + */ diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index 564b391d2d7e4..11119646f4a00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -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 - * - * 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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java index 2dfbba6467ac8..715eae7f4f4de 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java @@ -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 - * - * 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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 28eb72e1be681..7d19ea40f7c67 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java index 350f5a9f4490b..0ce799370e11a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -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 - * - * 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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 944b09d5720df..5006ee2dafae7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -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 org.apache.kafka.clients; import org.apache.kafka.common.protocol.SecurityProtocol; diff --git a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java index 18e7e182f559f..62ffada5d7836 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java @@ -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 - * - * 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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java index f4f753e495b6a..06b8fedad1348 100644 --- a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java +++ b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java @@ -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 - * - * 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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index f51d1f5131a0f..86ffa498bdbaf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -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 - * - * 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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java index 6486d1528d26b..ed149fb71059d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java @@ -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 org.apache.kafka.clients; import org.apache.kafka.common.Node; diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 75d48abf247ac..65da33054962a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java index 34bdbf6cbbcfc..55901b5df0144 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java @@ -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 org.apache.kafka.clients; import org.apache.kafka.common.Node; diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 79d467af0212e..1374a776dadb4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java index 0fcdf4d39bb4c..f216c60d1401b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java @@ -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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java index 6fee4e45986d2..4e08ddf328c3e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java @@ -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 - * - * 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 org.apache.kafka.clients; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java index 5695be83d6cbc..c6006b7ea80d7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitFailedException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index cdcab5d4ce4b9..b1badefd31802 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 1b335173b8884..26a7d5daffc7f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerInterceptor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerInterceptor.java index f8789fc57e9be..2f4e310f7025e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerInterceptor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerInterceptor.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java index a4265abb76eda..3a3873a83d1ef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceListener.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index 5f1015512285c..26c37680bed96 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -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 - * - * 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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index 5b83f0c677c66..f2dc9bbc2ba31 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -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 - * - * 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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/InvalidOffsetException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/InvalidOffsetException.java index 5f8a57faad31c..b23ca867359ee 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/InvalidOffsetException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/InvalidOffsetException.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index a90303632a436..63a39faf4481d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 9f312a71de5a5..d81270adf650a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java index 70fba3690b7e9..14bb71003379b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java @@ -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 - * - * 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 org.apache.kafka.clients.consumer; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java index df8bf37adbada..9d06f29721ffd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java index 2f95291112ac2..3af057f9ce4ae 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetCommitCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetCommitCallback.java index 918087d4201b7..2fef79ee2bbdb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetCommitCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetCommitCallback.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetOutOfRangeException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetOutOfRangeException.java index 3dd92fb0e74af..dae19b29f2d19 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetOutOfRangeException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetOutOfRangeException.java @@ -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 - * - * 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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java index 542da7f7bf5c8..6d742b850a134 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetResetStrategy.java @@ -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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java index 16c1d77c429a7..ec6c62f1e834e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RetriableCommitFailedException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RetriableCommitFailedException.java index 1c1a2f513cc56..33cba6fbc8d66 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RetriableCommitFailedException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RetriableCommitFailedException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java index a598258e339a6..8e38b84ccef88 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java @@ -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 - *

    - * 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 org.apache.kafka.clients.consumer; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 33061487966d1..db665b6788f2f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -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 - *

    - * 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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java index 4f90e66f2794c..5c97693e91fc7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java @@ -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 - *

    - * 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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 12ff9ce699113..dad59ea92e1e5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptors.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptors.java index f22686ec1843a..d96d8ce82a790 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptors.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptors.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 5a9778ce2133b..8781676cba467 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java index 361865d0082fc..392e27289d206 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index d67b9402a25db..8a8952cb125b6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -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 org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientResponse; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index dff100601034b..38a7c78a599c4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -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 - * - * 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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java index 0ec6017b345db..d1ad6a48ba502 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceListener.java index 3cb152de92dc4..a3acc83471314 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceListener.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceListener.java @@ -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 - * - * 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 org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java index 02eddd1eede50..86683a0fa7631 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java index 3a55cedeb7ca9..2b7c8f30e01ff 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java index cc5322ff0b57a..7261c0fea143c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java index b39261b70b51f..3a624ebf503a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFutureListener.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java index 09114cb33745b..53110d3838747 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StaleMetadataException.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 25995fb5c04c2..421a3cf983897 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/BufferExhaustedException.java b/clients/src/main/java/org/apache/kafka/clients/producer/BufferExhaustedException.java index f375f8f715bf3..929b6b9cc6619 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/BufferExhaustedException.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/BufferExhaustedException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java index 7caefc33603ea..a70e4e9a68530 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index d8d94e6aaadf3..5f83e40ab9ba7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -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 org.apache.kafka.clients.producer; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 165437de1085e..bafb04839fd00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java index c973cd3daf7da..a3a2fbea86408 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Partitioner.java @@ -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 org.apache.kafka.clients.producer; import org.apache.kafka.common.Configurable; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index d4a5d391bf23f..a77ecd08eaaee 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 89a2a6f93d293..70b80c776938c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -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 org.apache.kafka.clients.producer; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java index 96643d36cc76d..f466547f4fce6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java index 86112f1d070ff..df896160f3521 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java @@ -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 - * - * 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 org.apache.kafka.clients.producer; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java index 00a4374ef5543..db3448723f036 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index 92d59d9b82568..a036faf8e8a0e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java index f8ed0b4cfa149..086534a27ed5d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java index 18088c1c04291..e9801733d2342 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java @@ -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 org.apache.kafka.clients.producer.internals; import org.apache.kafka.clients.producer.Callback; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java index 1cde13d1b2d6e..f8b38e81052a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java @@ -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 - * - * 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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java index a98b28e52728c..0a73c417eb001 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java index 8466d3a85752d..da3be01ebbe99 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 800a85794194f..7ef28e5031006 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -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 - * - * 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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index 6346fe9c9f033..af6262ec63fce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -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 - * - * 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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 5483c4a09e717..b3553afd8ba42 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -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 - * - * 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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 3c6475d4ff0dd..b7408e3ad6bfd 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -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 - * - * 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 org.apache.kafka.common; diff --git a/clients/src/main/java/org/apache/kafka/common/ClusterResource.java b/clients/src/main/java/org/apache/kafka/common/ClusterResource.java index 29a87d4cc5064..c2737c9241805 100644 --- a/clients/src/main/java/org/apache/kafka/common/ClusterResource.java +++ b/clients/src/main/java/org/apache/kafka/common/ClusterResource.java @@ -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 org.apache.kafka.common; diff --git a/clients/src/main/java/org/apache/kafka/common/ClusterResourceListener.java b/clients/src/main/java/org/apache/kafka/common/ClusterResourceListener.java index 71de5341dd1fd..f8f99ece18384 100644 --- a/clients/src/main/java/org/apache/kafka/common/ClusterResourceListener.java +++ b/clients/src/main/java/org/apache/kafka/common/ClusterResourceListener.java @@ -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 org.apache.kafka.common; /** diff --git a/clients/src/main/java/org/apache/kafka/common/Configurable.java b/clients/src/main/java/org/apache/kafka/common/Configurable.java index 98774cc8dd4eb..ecca298ef793e 100644 --- a/clients/src/main/java/org/apache/kafka/common/Configurable.java +++ b/clients/src/main/java/org/apache/kafka/common/Configurable.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/KafkaException.java b/clients/src/main/java/org/apache/kafka/common/KafkaException.java index 124ef1ee2e7e2..7a20691210b5e 100644 --- a/clients/src/main/java/org/apache/kafka/common/KafkaException.java +++ b/clients/src/main/java/org/apache/kafka/common/KafkaException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/Metric.java b/clients/src/main/java/org/apache/kafka/common/Metric.java index d4ef77e36bbea..908c5c7a45307 100644 --- a/clients/src/main/java/org/apache/kafka/common/Metric.java +++ b/clients/src/main/java/org/apache/kafka/common/Metric.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/MetricName.java b/clients/src/main/java/org/apache/kafka/common/MetricName.java index 555b4e6aa025d..cd1ae4e529646 100644 --- a/clients/src/main/java/org/apache/kafka/common/MetricName.java +++ b/clients/src/main/java/org/apache/kafka/common/MetricName.java @@ -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 org.apache.kafka.common; @@ -205,4 +209,4 @@ public String toString() { return "MetricName [name=" + name + ", group=" + group + ", description=" + description + ", tags=" + tags + "]"; } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/common/Node.java b/clients/src/main/java/org/apache/kafka/common/Node.java index f569ddd576aae..8187369ca3c59 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -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 org.apache.kafka.common; diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java index 73109d47f75b8..b35111696373c 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -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 - * - * 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 org.apache.kafka.common; diff --git a/clients/src/main/java/org/apache/kafka/common/TopicPartition.java b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java index 383c00d4a1cb6..832bcd8b21ee6 100644 --- a/clients/src/main/java/org/apache/kafka/common/TopicPartition.java +++ b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java b/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java index 0d38f56864587..ce26c60001dcf 100644 --- a/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java +++ b/clients/src/main/java/org/apache/kafka/common/annotation/InterfaceStability.java @@ -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 org.apache.kafka.common.annotation; diff --git a/clients/src/main/java/org/apache/kafka/common/cache/Cache.java b/clients/src/main/java/org/apache/kafka/common/cache/Cache.java index 6c81fafca8a3e..0da4907920565 100644 --- a/clients/src/main/java/org/apache/kafka/common/cache/Cache.java +++ b/clients/src/main/java/org/apache/kafka/common/cache/Cache.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.common.cache; /** diff --git a/clients/src/main/java/org/apache/kafka/common/cache/LRUCache.java b/clients/src/main/java/org/apache/kafka/common/cache/LRUCache.java index 39576a6e71c8b..bdc67ac2732a3 100644 --- a/clients/src/main/java/org/apache/kafka/common/cache/LRUCache.java +++ b/clients/src/main/java/org/apache/kafka/common/cache/LRUCache.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.common.cache; import java.util.LinkedHashMap; diff --git a/clients/src/main/java/org/apache/kafka/common/cache/SynchronizedCache.java b/clients/src/main/java/org/apache/kafka/common/cache/SynchronizedCache.java index 0e88aa3e12de3..27cc4ba530c14 100644 --- a/clients/src/main/java/org/apache/kafka/common/cache/SynchronizedCache.java +++ b/clients/src/main/java/org/apache/kafka/common/cache/SynchronizedCache.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.common.cache; /** diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index bc852bd81c0b1..386ba31477bd2 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -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 org.apache.kafka.common.config; diff --git a/clients/src/main/java/org/apache/kafka/common/config/Config.java b/clients/src/main/java/org/apache/kafka/common/config/Config.java index 26f21d4f816b6..f7fa95c1de129 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/Config.java +++ b/clients/src/main/java/org/apache/kafka/common/config/Config.java @@ -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 org.apache.kafka.common.config; diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 25de8536d35fb..f7cb8a9eeafdb 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -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 org.apache.kafka.common.config; diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigException.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigException.java index 9a7fdf5996e23..c48bfc6c07e51 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigException.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java index fea51b719d83b..dafd7c657c5c8 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.common.config; import java.util.ArrayList; diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java index 9ae69e11edf05..7dc501251b62a 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -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 org.apache.kafka.common.config; import java.util.Collections; diff --git a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java index a1a9ab7800e95..8cae00d970a20 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java @@ -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 org.apache.kafka.common.config; import javax.net.ssl.KeyManagerFactory; diff --git a/clients/src/main/java/org/apache/kafka/common/config/types/Password.java b/clients/src/main/java/org/apache/kafka/common/config/types/Password.java index db8f82173e708..eafffb967f017 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/types/Password.java +++ b/clients/src/main/java/org/apache/kafka/common/config/types/Password.java @@ -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 org.apache.kafka.common.config.types; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java index a566b9006a63c..aa4e98c1b7b6b 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java index 7b60e11cd0d4d..aa4a111a00cb0 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java index d8348cf7711f3..0471fe643d9a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/AuthorizationException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/BrokerNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/BrokerNotAvailableException.java index f78f0619ed9fc..26bb803e84dd3 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/BrokerNotAvailableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/BrokerNotAvailableException.java @@ -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 org.apache.kafka.common.errors; public class BrokerNotAvailableException extends ApiException { diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ClusterAuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/ClusterAuthorizationException.java index 9b8c74d0673a1..61b892984863c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/ClusterAuthorizationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ClusterAuthorizationException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ControllerMovedException.java b/clients/src/main/java/org/apache/kafka/common/errors/ControllerMovedException.java index 8dd7487af2e57..124e79353ff3c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/ControllerMovedException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ControllerMovedException.java @@ -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 org.apache.kafka.common.errors; public class ControllerMovedException extends ApiException { diff --git a/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java index 56fa4b4c29758..1eac62196b8f6 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java index 557681cade8de..e0bc787ad55b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/DisconnectException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupAuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupAuthorizationException.java index 3a767aa5438b3..c3f0795de43e5 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/GroupAuthorizationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/GroupAuthorizationException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java index 554b8856d57da..3409b68f0e9d7 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java @@ -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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java index e227ca2cec963..f579a37881412 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java @@ -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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java index fe8ba7a4bd38b..efd749f7f2c95 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/IllegalGenerationException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java b/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java index 7fd008c1bf7dd..c45f00776fb28 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InconsistentGroupProtocolException.java b/clients/src/main/java/org/apache/kafka/common/errors/InconsistentGroupProtocolException.java index f3ccce0191b99..28bcbe5a962de 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InconsistentGroupProtocolException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InconsistentGroupProtocolException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java b/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java index 3680f1b658ba5..fec66bb0ec821 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidCommitOffsetSizeException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidCommitOffsetSizeException.java index 9573ad21df6ca..a17a30fab8586 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidCommitOffsetSizeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidCommitOffsetSizeException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java index 25cbc7ae6750e..333566a64ba1f 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidConfigurationException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidFetchSizeException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidFetchSizeException.java index 230f5a7329b94..65a0aebbc3677 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidFetchSizeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidFetchSizeException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidGroupIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidGroupIdException.java index 191f456ebef9e..95e6f36ea030b 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidGroupIdException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidGroupIdException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java index 8841badb2d783..e6663dbf185d0 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidOffsetException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidOffsetException.java index 135213d9dd031..0d954f1268a9d 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidOffsetException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidOffsetException.java @@ -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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidPartitionsException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidPartitionsException.java index c7ea66801802d..c65ced43cf79c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidPartitionsException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidPartitionsException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicaAssignmentException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicaAssignmentException.java index 765f0f6726ed9..5357d9194642f 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicaAssignmentException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicaAssignmentException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java index 33f048a532870..699d5a83a432a 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidReplicationFactorException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequestException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequestException.java index 8299da45e7b2a..7470f663f4406 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequestException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequestException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java index 9d19b2844aec7..423c09166b7ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidSessionTimeoutException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidSessionTimeoutException.java index 3ee411e3c266a..a971498426f12 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidSessionTimeoutException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidSessionTimeoutException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java index d2d285bc97412..0e3cd929ef77b 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java @@ -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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java index 1d90b592d1641..5c7b2be8f108d 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTopicException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java index bfdd4b7a559e8..69bc624e5a82a 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java index f0baa983f081e..fadd9bd484789 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotControllerException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotControllerException.java index c2784a880e8f5..1c3e01457f763 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotControllerException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotControllerException.java @@ -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 org.apache.kafka.common.errors; public class NotControllerException extends RetriableException { diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java index bc56eb03e81cc..d2ffaea588d8e 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java @@ -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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java index fd7f6d8f5c69b..22ebc3407d5ee 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java index 1573227acef53..cb90e8690320f 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java index ad9c77c41c492..718277905751d 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java index 66b4fff790e5d..b77f16740b32f 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java index 6b7a39d59c40f..92a70fdfed3dd 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java @@ -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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java b/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java index 393a6df4bf6dd..0316938c861c4 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java @@ -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 org.apache.kafka.common.errors; /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RebalanceInProgressException.java b/clients/src/main/java/org/apache/kafka/common/errors/RebalanceInProgressException.java index ee7aa27b6c9dc..031abb7f7bb3e 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RebalanceInProgressException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RebalanceInProgressException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java index f3f3f27d34dfb..b1ef77d4b93a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java index d84444564bdfe..9ffaa876a43c1 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ReplicaNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/ReplicaNotAvailableException.java index d0338faab4c93..b94d400d8fe31 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/ReplicaNotAvailableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ReplicaNotAvailableException.java @@ -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 org.apache.kafka.common.errors; public class ReplicaNotAvailableException extends ApiException { diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java b/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java index 6c639a972d7e4..213e4f9fe0844 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java b/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java index 00388d12794f0..76c87777c4c42 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java index c7f569ca87369..47fe034701bfd 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TopicAuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/TopicAuthorizationException.java index b5d33b9e3de75..6bf260d5cde79 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/TopicAuthorizationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/TopicAuthorizationException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TopicExistsException.java b/clients/src/main/java/org/apache/kafka/common/errors/TopicExistsException.java index 0fc0683195229..cc0c8f14c818b 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/TopicExistsException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/TopicExistsException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java index f8eab90650f88..f6eea5bc8ea02 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java index 963ef081db51d..37e003bfef6d1 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java index 70fd55c8788be..0f2a56268908a 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java @@ -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 - * - * 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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java index 00356f3a7cb89..7992d107dce8a 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedForMessageFormatException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java index 289a09f78dae7..9dab22ac7a07c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java index 3679be4fc31be..17bd71e3b3be3 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/WakeupException.java b/clients/src/main/java/org/apache/kafka/common/errors/WakeupException.java index a2e718d2f9ed2..4726ec13cca94 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/WakeupException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/WakeupException.java @@ -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 org.apache.kafka.common.errors; diff --git a/clients/src/main/java/org/apache/kafka/common/internals/ClusterResourceListeners.java b/clients/src/main/java/org/apache/kafka/common/internals/ClusterResourceListeners.java index cab29ccb256df..1209f38453f22 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/ClusterResourceListeners.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/ClusterResourceListeners.java @@ -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 org.apache.kafka.common.internals; import org.apache.kafka.common.ClusterResource; diff --git a/clients/src/main/java/org/apache/kafka/common/internals/FatalExitError.java b/clients/src/main/java/org/apache/kafka/common/internals/FatalExitError.java index 4f23c4e69b157..fd6810fb765d0 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/FatalExitError.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/FatalExitError.java @@ -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 org.apache.kafka.common.internals; diff --git a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java index 49823c099c6bc..605372c6ec7e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java @@ -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 org.apache.kafka.common.internals; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java index e0969aad858e4..69f3efafe2fc0 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java index 6872049dae7f1..52704d63071b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java @@ -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 - * - * 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 org.apache.kafka.common.metrics; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java index 86014e53bb892..1cd5b24333941 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java b/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java index 79f61bca4c3fa..d068895fee2c0 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java @@ -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 - * - * 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 org.apache.kafka.common.metrics; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MeasurableStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/MeasurableStat.java index 27cc0db522a5e..aedac9a764c82 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MeasurableStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MeasurableStat.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java index 0766cc34f942a..1fffee72c168a 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java index 512c18e74aca2..3861d30056626 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java @@ -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 - * - * 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 org.apache.kafka.common.metrics; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java index ab75813b2fa49..995bdaa6dce3a 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java @@ -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 - * - * 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 org.apache.kafka.common.metrics; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java index 663b963afdd6c..e4141339439f1 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java index 7e1a2c64aa9fe..4c970f26aea87 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index e85b55915cb35..5ca9fce2c0c52 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -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 - * - * 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 org.apache.kafka.common.metrics; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java b/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java index d4cfa39471b88..1ddf5be2ec514 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java index a65dd287f4afd..a09ca5af3e2ca 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java index 9ca3231ea77b3..3da91c4e908c6 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java index c571b4b717660..7d4123e8e01ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java index f0d741c965274..6d75454a9ccb1 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java index 5cfb535f460ab..7a18a2d0ea44d 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java index fb741ae086c71..f8ae3a17601a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java index df0ee4a0288d4..c4c8d3986478b 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index c0dd10e6e459e..a56734cd86dfb 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java index 58ea93f99c96a..18dd6f271b076 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java index 018f0848b4d98..69d44a8b2de22 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java index 67999a90ab9fe..b8a83f5004bd6 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java @@ -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 - * - * 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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Value.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Value.java index d83a023a4723a..deb81c7b7cb9a 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Value.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Value.java @@ -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 org.apache.kafka.common.metrics.stats; diff --git a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java index 0012f158f87f3..b242967d9527b 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java @@ -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 org.apache.kafka.common.network; import java.io.Closeable; diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java index 307eadf8db6c9..5b0af02cb6155 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java index a7ba6bf153bc2..23a169f4bc4d9 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java index 0b7c328d5c8d1..303d76fa2c363 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java index 02fb5e80d4acf..5145be7f29511 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java @@ -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 org.apache.kafka.common.network; import org.apache.kafka.common.config.AbstractConfig; diff --git a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java index 650ad4151bc9f..77d94a615e329 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java @@ -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 org.apache.kafka.common.network; import java.security.Principal; diff --git a/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java b/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java index a5bdd62240c24..a56353a420208 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java +++ b/clients/src/main/java/org/apache/kafka/common/network/InvalidReceiveException.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java index 9b05aeb63eea0..f1bf86c7114ec 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java +++ b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/ListenerName.java b/clients/src/main/java/org/apache/kafka/common/network/ListenerName.java index fad2ee27330e9..1245b1fe9fedd 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ListenerName.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ListenerName.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/network/Mode.java b/clients/src/main/java/org/apache/kafka/common/network/Mode.java index 59ef712da9ba7..6123970324916 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Mode.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Mode.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java b/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java index 0285fe376cfd6..f77ff971e8039 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/MultiSend.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java index 409775cd9785c..582f064639184 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java b/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java index 42015795f0c8f..8820059aa7985 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java index c573672e1043c..fb4f6ba73a1cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java index 1135359f8d196..871b7ac661673 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java @@ -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 org.apache.kafka.common.network; /* diff --git a/clients/src/main/java/org/apache/kafka/common/network/Receive.java b/clients/src/main/java/org/apache/kafka/common/network/Receive.java index 57599f7a8a414..b7bbdb4b180b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Receive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Receive.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java index 060f83308f12f..6b5668f1360ad 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java index 10ca632357ae7..6eca4270afb86 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index df3526618cc5d..fd3ab47afcd2a 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/Send.java b/clients/src/main/java/org/apache/kafka/common/network/Send.java index 628fb29acf4d7..c64193aff4dc1 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Send.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Send.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java index 1d612bd14d6da..8e55d48410c39 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslChannelBuilder.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java index 9ccf33dd40e9b..eb423e338f5a3 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java @@ -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 org.apache.kafka.common.network; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java index 092df4dbb0bd2..fad0cea690eac 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java @@ -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 org.apache.kafka.common.network; /* diff --git a/clients/src/main/java/org/apache/kafka/common/network/TransportLayers.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayers.java index b77eae8d05bbe..f5bc1fc7fda40 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/TransportLayers.java +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayers.java @@ -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 org.apache.kafka.common.network; import java.nio.channels.GatheringByteChannel; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 4e525df00d60d..b3c59a118bfad 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index e7689e2e11908..a2a33eee0a4ed 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 032e00e0a158f..25d380b6df4bb 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java index fe48832b49b9d..99d3b3d518a12 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java index 1ba8e44b5fa75..4213ecd08f054 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java index cab7bf47dd3f2..605174de2dde1 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java index efbfd1416c393..f275ada664f36 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java @@ -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 - * - * 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 org.apache.kafka.common.protocol.types; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java index 58b685bd61bc7..48a3bdc5f777f 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 0165ce685288e..1258763d43477 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -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 org.apache.kafka.common.protocol.types; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java index e2f7bafc4e140..3341f3ef69329 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java index 47b96e2f12eb7..1548a95443a7a 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import java.util.ArrayList; diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java index fca45cf7f4b61..c033b6c70bec8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java index ee7d3089a2c68..bdda998909b6c 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.apache.kafka.common.errors.CorruptRecordException; diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java index 13609d8baa1f0..4eee605e96c0a 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java index 62265dc77d7fa..658e50ca6a8cf 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java index dd9cc84dfb253..1461b55ccaabe 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index 960b7168ff70f..0a333c5d80166 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; diff --git a/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java b/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java index ee6071314ff60..ffd09a471b1f5 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java +++ b/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java index a408580271b0a..6544d13b1347d 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java @@ -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 org.apache.kafka.common.record; import static org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK; diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java index 515c09a1263c8..0a64d43e6344f 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java @@ -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 org.apache.kafka.common.record; import java.io.FilterOutputStream; diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java index d2db3569f0ef3..d4ffec08397cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java index a9af651d4d079..d81488ddf47bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 6c31b25298f1a..c5076d6285cb1 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -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 - * - * 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 org.apache.kafka.common.record; diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 731d1dc25778d..32ddcee1acdec 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java index e3d0b76ce9020..a54d65dfd8718 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Record.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/Records.java b/clients/src/main/java/org/apache/kafka/common/record/Records.java index bdc96558b7930..5e0490dd4a501 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Records.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Records.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java index 430db30c6c8e2..b6f049c67c351 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; diff --git a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java index 182cbd1dcfd01..9440668137517 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java @@ -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 org.apache.kafka.common.record; import java.util.NoSuchElementException; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index e2403bc13cb9e..c2ad39db3deb7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java index 00ddf71dc5737..0ba373d6fea7a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index b018871843901..d534daf7f1ed4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.requests; import org.apache.kafka.common.network.NetworkSend; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java index fde07bfc404de..07dd5f518fb3a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 6fea9791ac5b3..382da89ac5070 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java index 48f0a11b5da1d..4b5ec13bf80c2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java index a407c5a6bf804..00973f00b35cc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java index 7e196878301ab..673810d5b0651 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java index f0986ffc29b61..33a4b4a76de90 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java index 1cb6eb887ac96..ccbe2110e4a30 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java index dc651e1a09f75..9d0d0f3ab21a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java index d5bde068b89b1..287eda951818f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java index bab13ffabeccb..5496d66867878 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 7418b8977e51e..9c33f71f4db5e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 25631a5ed921d..79b1a5cd0cdc5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java index dcd71567166a8..b45054c85d2c1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java index 8a0f5cdfcbc3c..f8a9f8f8a0aac 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupCoordinatorResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 0b713de2c3fd8..44591a08c720f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index ac58a70237f4b..18d63f829016c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 93f124ae07d34..994d9a25d6d72 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index 82f64456fba7a..bd55e9b3e52bb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index 942590e5844f1..f51cfa918b84e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.Node; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java index 3ec1d0534fee3..bc4400eb727bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java index f89675eaa01b4..4b5820b1fc795 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java index 983027e58cdf6..49b704b9ea178 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java index cca5399ce12d1..cceff927f4fd1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index ae54d33dc3812..f2ac33bccceec 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index aea16de9a18b6..1d62a963d9182 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index d4ab2d2982141..3f049b427a0eb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index db85c34e1ad64..98d57eddcde62 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index fa79e92c3442d..02dbe392cb66c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 556292ca8fcc1..6459201ad3f5f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 8c0ff97575e2a..b30505b89f7c8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 653cbd864e655..71ec3f6847337 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 189575e2341a4..5e69bebad0925 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java index e766632162833..035b33043ca0b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java @@ -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 org.apache.kafka.common.requests; import java.util.Arrays; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 7b454e867c3a9..5e4b5fbc67fb1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 823473226e329..cae6e94b4ed01 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -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 - * - * 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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RecordsSend.java b/clients/src/main/java/org/apache/kafka/common/requests/RecordsSend.java index f1b6b25a7af77..6608e9b121823 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RecordsSend.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RecordsSend.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.requests; import org.apache.kafka.common.network.Send; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java index d2147b3181400..4f94a09f17ac3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestAndSize.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java index 5e65132449ff3..4b818e71cbdbf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java @@ -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 - * - * 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 org.apache.kafka.common.requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java index 04390eaae34eb..895716bd334b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java index 9cc3f1fa008b6..56c29c5ab5e8d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.requests; import java.nio.ByteBuffer; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java index e1a4c87caa3a8..e70bad38550cf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.requests; import java.nio.ByteBuffer; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java index 7b79cd8e5d25e..325ee06c009f9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java index 617b1c6a46acc..c859f7ff0104b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java index b55ccff17b435..dbc19ac24c390 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java index 148815c54f9da..5d50c5cc637ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index 98bf83b3cfa9d..41b0c84c1f07d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java index 5b8b46d2f2dda..4eae39e8f3c91 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.protocol.ApiKeys; diff --git a/clients/src/main/java/org/apache/kafka/common/security/JaasConfig.java b/clients/src/main/java/org/apache/kafka/common/security/JaasConfig.java index d900be29780ae..e0bcffe85d915 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/JaasConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/security/JaasConfig.java @@ -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 org.apache.kafka.common.security; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/JaasContext.java b/clients/src/main/java/org/apache/kafka/common/security/JaasContext.java index 6abeef62b5712..a13acd2dc6d4a 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/JaasContext.java +++ b/clients/src/main/java/org/apache/kafka/common/security/JaasContext.java @@ -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 org.apache.kafka.common.security; import org.apache.kafka.common.config.SaslConfigs; diff --git a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java index ca6b7f082aff6..08f41c3349be5 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java @@ -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, diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java index ed2c087fdfc82..bfee5773673b9 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java @@ -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 org.apache.kafka.common.security.auth; import java.util.Map; diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java index 0ea935c58a650..04e4c90a7403e 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java @@ -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 org.apache.kafka.common.security.auth; import java.util.Map; diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java b/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java index 06c59d1c08317..4bacdcbb355a2 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java b/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java index bc550943b6397..72b5725addef8 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.auth; import org.apache.kafka.common.security.JaasContext; diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java index 75e18555e331b..fc2a930f0b536 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java @@ -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 org.apache.kafka.common.security.auth; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java index 1b9953cb50937..18862be3d663b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.authenticator; import javax.security.auth.login.LoginContext; diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java index 568dcb5f3010c..aa39101753fe0 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.authenticator; import java.util.HashMap; diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java index 0a405bc3f2be3..a902d7dfc4d29 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.authenticator; public class DefaultLogin extends AbstractLogin { diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java index 55b561cc80384..1730d660a5c6d 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.authenticator; import javax.security.auth.Subject; diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java index 27a24e5bf09a4..149148e306848 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.authenticator; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java index 3a38e24b94b61..3391ff3436026 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java @@ -1,13 +1,12 @@ -/** - * 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, diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index 88f8959282611..c249b68200a53 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -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 org.apache.kafka.common.security.authenticator; import org.apache.kafka.common.KafkaException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java index 94083fb8b117b..52a34fc4a83d7 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.authenticator; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/BadFormatString.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/BadFormatString.java index 4d49a83c5baae..3f9070b147a98 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/BadFormatString.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/BadFormatString.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.kerberos; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java index 23d163cf621dd..1a579ac55cd34 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.kerberos; import javax.security.auth.kerberos.KerberosPrincipal; diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java index 4f8e097716e95..8ac6d7eda6b68 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.kerberos; import java.util.regex.Matcher; diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java index b366f1b78a9b0..40a9a1fad6e54 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.kerberos; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java index 63538c75fd033..1db984a6cb744 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosShortNamer.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.kerberos; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java index 6c2d2677c31d7..387c2228d52c8 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.kerberos; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java index f06fbf62fb387..c8b29fc69a9cc 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java +++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.plain; import java.util.Map; diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java index 1ad02237be3b8..df59820f03ac2 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java +++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.plain; import java.io.UnsupportedEncodingException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java index c3db1f577411b..51998a9ebc481 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java +++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.plain; import java.security.Provider; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredential.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredential.java index 7de48f2261c2e..09ff0aacd3c84 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredential.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredential.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; public class ScramCredential { diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialCallback.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialCallback.java index 07c1c93d9cab8..931210a0e0702 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialCallback.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialCallback.java @@ -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 org.apache.kafka.common.security.scram; import javax.security.auth.callback.Callback; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialUtils.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialUtils.java index 359e46d603eb4..8120c156b4d3f 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramCredentialUtils.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.util.Collection; @@ -83,4 +81,4 @@ public static void createCache(CredentialCache cache, Collection enabled cache.createCache(mechanism, ScramCredential.class); } } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramFormatter.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramFormatter.java index e600573abea8c..406c28599b3bb 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramFormatter.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramFormatter.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.math.BigInteger; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramLoginModule.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramLoginModule.java index 1e4b643103acd..c087a3215d782 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramLoginModule.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramLoginModule.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.util.Map; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMechanism.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMechanism.java index 86b51bca77e10..d8c0c6d3d603e 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMechanism.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMechanism.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.util.Collection; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMessages.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMessages.java index 3f94e657cd0ac..6fd117dcee3e4 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMessages.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramMessages.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.nio.charset.StandardCharsets; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClient.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClient.java index 4220a69ec9ab0..e5b0f84744b51 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClient.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClient.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClientProvider.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClientProvider.java index aa5fc5681bd2b..fac673e6dd333 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClientProvider.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslClientProvider.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.security.Provider; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServer.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServer.java index 4298f98b83629..942c299979866 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServer.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServer.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServerProvider.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServerProvider.java index 496cb5f9a0bcf..2f768a35fe7f8 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServerProvider.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramSaslServerProvider.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.common.security.scram; import java.security.Provider; diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramServerCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramServerCallbackHandler.java index 002489c5983e6..f81c7f1911d79 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/ScramServerCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/ScramServerCallbackHandler.java @@ -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 org.apache.kafka.common.security.scram; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java index ca49a3be43a37..77f75f61864dc 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java index d89b3ff0509eb..267211576b6d0 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java @@ -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 org.apache.kafka.common.serialization; import java.util.Map; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java index beaef948e38f2..d069e9495e6ec 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java @@ -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 org.apache.kafka.common.serialization; import java.util.Map; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferDeserializer.java index 90c1ba0a3b72e..d41f03c6675ee 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferDeserializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferSerializer.java index a9547059a2eec..c8c369272ddc3 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteBufferSerializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java index ee6a57cb19e72..66b07eb58419e 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java index 3d0444668d3a1..0dc4476d46d32 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java index 95f6fd72b4560..383f6e345b06a 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java @@ -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 org.apache.kafka.common.serialization; import java.io.Closeable; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java index ed4f323791d82..24f6007cb35fc 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java index 9d01342c59625..7dd4edc3b62ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/IntegerDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerDeserializer.java index 0b5a58d858401..29c3accb4b864 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/IntegerDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerDeserializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/IntegerSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerSerializer.java index 578bdd2f673a9..f2144ceee707c 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/IntegerSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/IntegerSerializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java index 37983e4a4c031..e5bfe3cd03c1e 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java index 31005299e7a41..d37842c391496 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java index 42b8c1e7f6379..fbcc7c2e0fcae 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java @@ -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 org.apache.kafka.common.serialization; import java.io.Closeable; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index e5fb6c0a3b8a7..15f9748e246e1 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -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 org.apache.kafka.common.serialization; import org.apache.kafka.common.utils.Bytes; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java index 5cfcc6317007f..233a6583bfd23 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java @@ -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 org.apache.kafka.common.serialization; import java.io.Closeable; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java index 9783ea0c53110..c647c9b73324d 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java index 636d905ffc729..c2862ddcb4e04 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java b/clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java index d6771a3ec0b7b..daf89e678216d 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java b/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java index 415ef2e673c17..1a4f77b39538d 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/AppInfoParser.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java index 78340e54daec4..e28d925acbc06 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java @@ -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 org.apache.kafka.common.utils; import java.util.Arrays; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CircularIterator.java b/clients/src/main/java/org/apache/kafka/common/utils/CircularIterator.java index 00be7834dcb7e..3e7d5eb29fbcc 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CircularIterator.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CircularIterator.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.common.utils; import java.util.Iterator; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java index 106a7d47f78b3..d7ab4e0b39caf 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java @@ -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 org.apache.kafka.common.utils; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java b/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java index 9c0e81ad80669..1a3351f7ab225 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java @@ -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 - * - * 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 org.apache.kafka.common.utils; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java index 48af070c97940..f779371c59df5 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java @@ -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 - * - * 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 org.apache.kafka.common.utils; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Exit.java b/clients/src/main/java/org/apache/kafka/common/utils/Exit.java index a1357fda8894d..7c2a663166e27 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Exit.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Exit.java @@ -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 org.apache.kafka.common.utils; /** diff --git a/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java index faca68574fb4c..3eb025b91f6cd 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java @@ -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 - * - * 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 org.apache.kafka.common.utils; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Shell.java b/clients/src/main/java/org/apache/kafka/common/utils/Shell.java index f5db5c333be29..3c765b59de8fb 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Shell.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Shell.java @@ -1,13 +1,12 @@ -/** - * 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, @@ -15,8 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.kafka.common.utils; import java.io.BufferedReader; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java index d331f9a01e0c0..60da06444d220 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Time.java b/clients/src/main/java/org/apache/kafka/common/utils/Time.java index c782619d6c4a9..c288bd33e54f1 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Time.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Time.java @@ -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 * diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index c8f57bb0e62bb..20ab814e49463 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -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 org.apache.kafka.common.utils; diff --git a/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java b/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java index 22a7c1d37ccb4..4ff0dc68acd90 100644 --- a/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java +++ b/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java @@ -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 org.apache.kafka.server.policy; import org.apache.kafka.common.Configurable; diff --git a/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java index 5128f9ed28f74..d19b0be4be49f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index cfd2a941eb9f5..1a05abc1e5b50 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -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 org.apache.kafka.clients; diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 7712d3c8577b2..94a7050c0aeb2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index eb6a25914e3d6..2fba771b9b2f5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java index 9f0868f202850..a2442444ef304 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java index 5a7699d0ea4b9..48580c2657d12 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java index 41e9160f1226b..9e622331ee3c0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.clients.consumer; import org.apache.kafka.common.record.TimestampType; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java index d68a341a55315..1f722a754a24c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.clients.consumer; import static org.junit.Assert.assertEquals; @@ -55,4 +54,4 @@ public void iterator() throws Exception { } assertEquals(2, c); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 5c4590b36e716..864700b0ae697 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index 70b1c0980ffb9..6ed46f711cbaf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java index 72febb02ca6ef..347e96af66b28 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java index 1d62700e5cbe1..ca413027277c1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java @@ -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 - *

    - * 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 org.apache.kafka.clients.consumer; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/SerializeCompatibilityOffsetAndMetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/SerializeCompatibilityOffsetAndMetadataTest.java index ce1d4cd955df4..324aeafd88646 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/SerializeCompatibilityOffsetAndMetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/SerializeCompatibilityOffsetAndMetadataTest.java @@ -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 org.apache.kafka.clients.consumer; import org.apache.kafka.common.utils.Serializer; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index afdecfc222ceb..3450f31bb7658 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.Metadata; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 33f496b2ca447..a89b67c00810d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java index 4259c751a7e52..571a1c3135304 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java index 83bb1453ddb3b..2a6b2286c9d24 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java index be98ce7e98980..37d105cf1cc3a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 6a17da801aac9..10bac59d76f13 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java index 0177c7915167f..06cdae7ad6942 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java index 40ae66165feda..a8e466458ccf6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java index bf3c712b68163..786408404d4e5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestFutureTest.java @@ -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 org.apache.kafka.clients.consumer.internals; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index 61a55e262f909..c0a2df96d2876 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index aee6fc7a49d07..53e0d08bd75cf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -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, diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index e3f86ef851ebc..a29b8819d59c9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java index a844bb0891701..7d725fa6aaa86 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java index bc8105bff5b6b..9eab4183ea5d4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java index 41143d874d161..b4a597c5c6b24 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java index 2e2c8210dc0e7..f3fdd656c919e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java @@ -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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java index 659a29f41ebd6..32b3ddbd45d8b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java @@ -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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index f8bb1e9942d19..b96594f2b8cf3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -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 - * - * 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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 6e054d05fd234..2210b6d31f73d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -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 org.apache.kafka.clients.producer.internals; diff --git a/clients/src/test/java/org/apache/kafka/common/ClusterTest.java b/clients/src/test/java/org/apache/kafka/common/ClusterTest.java index c4b5d4609ae4b..0a7049bcc8899 100644 --- a/clients/src/test/java/org/apache/kafka/common/ClusterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/ClusterTest.java @@ -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 org.apache.kafka.common; diff --git a/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java b/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java index 7c2de5701b379..7836023fa68d2 100644 --- a/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java +++ b/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java @@ -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 org.apache.kafka.common; diff --git a/clients/src/test/java/org/apache/kafka/common/SerializeCompatibilityTopicPartitionTest.java b/clients/src/test/java/org/apache/kafka/common/SerializeCompatibilityTopicPartitionTest.java index 7786a730a3e16..8b4df5ff01f6a 100644 --- a/clients/src/test/java/org/apache/kafka/common/SerializeCompatibilityTopicPartitionTest.java +++ b/clients/src/test/java/org/apache/kafka/common/SerializeCompatibilityTopicPartitionTest.java @@ -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 org.apache.kafka.common; import org.apache.kafka.common.utils.Serializer; diff --git a/clients/src/test/java/org/apache/kafka/common/cache/LRUCacheTest.java b/clients/src/test/java/org/apache/kafka/common/cache/LRUCacheTest.java index 4cf130ce09043..8dab7d998c0f8 100644 --- a/clients/src/test/java/org/apache/kafka/common/cache/LRUCacheTest.java +++ b/clients/src/test/java/org/apache/kafka/common/cache/LRUCacheTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.common.cache; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java index 00c604e2219e5..9e2117975a957 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java @@ -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 - * - * 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 org.apache.kafka.common.config; diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 5a6339ee64847..10271ca35682c 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -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 org.apache.kafka.common.config; diff --git a/clients/src/test/java/org/apache/kafka/common/internals/PartitionStatesTest.java b/clients/src/test/java/org/apache/kafka/common/internals/PartitionStatesTest.java index 66c7abc802471..65a812a77b172 100644 --- a/clients/src/test/java/org/apache/kafka/common/internals/PartitionStatesTest.java +++ b/clients/src/test/java/org/apache/kafka/common/internals/PartitionStatesTest.java @@ -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 org.apache.kafka.common.internals; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java b/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java index d5dd9b8b5631a..99dfc302acad7 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java @@ -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 - * - * 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 org.apache.kafka.common.metrics; diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java index e07e646334ed7..3f09e083b9b06 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 5797b3697588a..1ef98a2ec0525 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -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 - * - * 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 org.apache.kafka.common.metrics; diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java index 9aadc82eee023..d22111e128e25 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/SensorTest.java @@ -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 org.apache.kafka.common.metrics; diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java index a55cc3226c68e..aba5e7a5a93f4 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/network/CertStores.java b/clients/src/test/java/org/apache/kafka/common/network/CertStores.java index 6f108b5545645..9cd981f838bdb 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/CertStores.java +++ b/clients/src/test/java/org/apache/kafka/common/network/CertStores.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java index 6df7b93d33698..abcc07ad6fdac 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java +++ b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java index 78c08d5b3f5ca..a3859c1b6700f 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java index fb00e9ca02011..f44131ab0adb0 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java +++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index 2d023e02d8c31..adff4b22b9bb0 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java index 06ad810fe4168..476ddfbc297f7 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslSelectorTest.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index db911e8eee28c..1de67898a6646 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -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 org.apache.kafka.common.network; diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index 8917a226e0789..a4aeb25b6eae7 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -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 org.apache.kafka.common.protocol; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java index e198e739ac96d..4a96ac43d8390 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index 1633e89acd564..74c930290cef3 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java b/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java index 0fad9a4e008dc..6475009e730cc 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java index 68e2a63cc43a8..5a1d5f4d10232 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java index d7df27da96e77..7a4dc1e90a3b7 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; diff --git a/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java b/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java index aa50426d5c1ab..2c3b2fcda0a63 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java +++ b/clients/src/test/java/org/apache/kafka/common/record/KafkaLZ4Test.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java index 4c2a71e040206..e1dcae468b8e5 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.apache.kafka.test.TestUtils; diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index 9271a3fa9e120..bfe0a57bc06ff 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java index f785c725ac222..daf8a87b94796 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java index ca78fc9cbd5a3..4bb90cddfd716 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java index e7e2a3b52ffc4..c2627584725ed 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.common.record; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 1367ba152d1f2..095b6632716d8 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -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 org.apache.kafka.common.requests; import org.apache.kafka.common.Node; diff --git a/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java b/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java index 6040aa2e66459..8d98a113c3372 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java b/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java index 8c1c038030167..4cd5f5d517233 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java +++ b/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java @@ -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 org.apache.kafka.common.security; diff --git a/clients/src/test/java/org/apache/kafka/common/security/auth/KafkaPrincipalTest.java b/clients/src/test/java/org/apache/kafka/common/security/auth/KafkaPrincipalTest.java index 051ad04b107f7..db905ddd649cd 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/auth/KafkaPrincipalTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/auth/KafkaPrincipalTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 8a9353240ebc3..1aea83574ce69 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -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 org.apache.kafka.common.security.authenticator; diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java index 2923a5a618b66..8f9bce5b9d0bd 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java @@ -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 org.apache.kafka.common.security.authenticator; diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java index fb73d6958fd91..bdef6ef3a914e 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java @@ -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 org.apache.kafka.common.security.authenticator; diff --git a/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java index 01e1a1aa2823a..3fbd310351e87 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java @@ -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 org.apache.kafka.common.security.kerberos; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramCredentialUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramCredentialUtilsTest.java index 9acc16ee2736e..e9dd285f54e62 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramCredentialUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramCredentialUtilsTest.java @@ -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 org.apache.kafka.common.security.scram; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramFormatterTest.java b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramFormatterTest.java index bb88f8feac4db..371031bcb0d1c 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramFormatterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramFormatterTest.java @@ -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 org.apache.kafka.common.security.scram; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java index dddc4d3661525..53939ef4c7d4a 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java @@ -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 org.apache.kafka.common.security.scram; import org.junit.Before; diff --git a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java index f078e4db342b7..5546a55123f79 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java @@ -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 org.apache.kafka.common.security.ssl; diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index e4cd67881734a..50b4594241409 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -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 org.apache.kafka.common.serialization; diff --git a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java index 3f41084f3ba1e..96adbbea31c67 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java index c39c3cff64d4e..7f3132eff435e 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java index 8178f4c42dabb..af67937a1f5b8 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java @@ -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 org.apache.kafka.common.utils; diff --git a/clients/src/test/java/org/apache/kafka/common/utils/Serializer.java b/clients/src/test/java/org/apache/kafka/common/utils/Serializer.java index f30c0e1aec5fe..a902449550a28 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/Serializer.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/Serializer.java @@ -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 org.apache.kafka.common.utils; import java.io.ByteArrayInputStream; diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 194cad6807422..c3f69fa4b507c 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/DelayedReceive.java b/clients/src/test/java/org/apache/kafka/test/DelayedReceive.java index 3d982cca1979a..37d8f742769e0 100644 --- a/clients/src/test/java/org/apache/kafka/test/DelayedReceive.java +++ b/clients/src/test/java/org/apache/kafka/test/DelayedReceive.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/MetricsBench.java b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java index 64a0921d30370..379db2f526d1d 100644 --- a/clients/src/test/java/org/apache/kafka/test/MetricsBench.java +++ b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java @@ -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 - * - * 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 org.apache.kafka.test; diff --git a/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java index f7a47b1865ea2..cfb5f6ca19675 100644 --- a/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java +++ b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java @@ -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 - * - * 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 org.apache.kafka.test; diff --git a/clients/src/test/java/org/apache/kafka/test/MockClusterResourceListener.java b/clients/src/test/java/org/apache/kafka/test/MockClusterResourceListener.java index dc12a8350c36e..c8185a81f089a 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockClusterResourceListener.java +++ b/clients/src/test/java/org/apache/kafka/test/MockClusterResourceListener.java @@ -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 org.apache.kafka.test; import org.apache.kafka.common.ClusterResourceListener; diff --git a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java index 8bf6983b6d12f..08c8e74b6563c 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java b/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java index 9c7fca59596eb..9704206926027 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java +++ b/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/MockMetricsReporter.java b/clients/src/test/java/org/apache/kafka/test/MockMetricsReporter.java index eddaa273e3021..b5f3855034fe2 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockMetricsReporter.java +++ b/clients/src/test/java/org/apache/kafka/test/MockMetricsReporter.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/MockPartitioner.java b/clients/src/test/java/org/apache/kafka/test/MockPartitioner.java index 553240948a606..af6d6fdd01677 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockPartitioner.java +++ b/clients/src/test/java/org/apache/kafka/test/MockPartitioner.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java index 22a588d737743..2a0751efc8ee8 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/MockSelector.java b/clients/src/test/java/org/apache/kafka/test/MockSelector.java index 6f080b0de3b99..c1b2205d66b43 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSelector.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSelector.java @@ -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 org.apache.kafka.test; diff --git a/clients/src/test/java/org/apache/kafka/test/MockSerializer.java b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java index e1bf2bb414a46..0c597c8b72aee 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSerializer.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSerializer.java @@ -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 * diff --git a/clients/src/test/java/org/apache/kafka/test/TestCondition.java b/clients/src/test/java/org/apache/kafka/test/TestCondition.java index f78c91bf5a2ae..9087ad41a9b2e 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestCondition.java +++ b/clients/src/test/java/org/apache/kafka/test/TestCondition.java @@ -1,19 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - - package org.apache.kafka.test; /** diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index 3c172941f3b43..7b78d3e500057 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -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 org.apache.kafka.test; import org.apache.kafka.common.config.SslConfigs; diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 0cb32be916b2e..b5fada4082847 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java index f3487c07017de..344e365fd3f08 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.connector; import org.apache.kafka.connect.data.Schema; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java index a46babd352660..a8a5dabfc839c 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.connector; import org.apache.kafka.common.config.Config; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java index 3b887c19e9704..3f98c6ab22ce1 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.connector; /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java index 261134d2b4951..42b87cd37df06 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.connector; import java.util.Map; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index e052534a7a955..651b2eebff875 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Date.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Date.java index 842da66eb7596..7a6fb3f2c1bb0 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Date.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Date.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Decimal.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Decimal.java index e15f69836b026..bf6fb3a5a3ca7 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Decimal.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Decimal.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Field.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Field.java index 7dd32ce9167f0..3a8df17a91017 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Field.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Field.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import java.util.Objects; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java index 3313182762150..c234217acd003 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import java.util.List; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaAndValue.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaAndValue.java index 71198f0a176a4..b9a539e41cd11 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaAndValue.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaAndValue.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import java.util.Objects; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java index 2ec1cf3d2a1a3..a5f2eda6c010b 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java index 6277e44c4e0d2..ea31752f60e58 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaProjector.java @@ -1,18 +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 org.apache.kafka.connect.data; import org.apache.kafka.connect.data.Schema.Type; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java index 200a1c00a9df5..6e7b5d23bbd78 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Time.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Time.java index cecd891ecd56c..b403ac11ee12c 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Time.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Time.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Timestamp.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Timestamp.java index cd7ed4a400768..8c426f4db3283 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Timestamp.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Timestamp.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/AlreadyExistsException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/AlreadyExistsException.java index 6fdefdf443b8b..a37f6157e7ca2 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/AlreadyExistsException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/AlreadyExistsException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.errors; /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java index bdddca03cbc96..3cbde36032edb 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.errors; import org.apache.kafka.common.KafkaException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/DataException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/DataException.java index 75426a382df63..a850347b2bbf7 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/DataException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/DataException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.errors; /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/IllegalWorkerStateException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/IllegalWorkerStateException.java index 8310059e0a256..be9cd34e4e83d 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/IllegalWorkerStateException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/IllegalWorkerStateException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.errors; /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/NotFoundException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/NotFoundException.java index a3bbe9105bfde..90f017903cab1 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/NotFoundException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/NotFoundException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.errors; /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/RetriableException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/RetriableException.java index 1b5b07a8accec..0b34bd0e14986 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/RetriableException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/RetriableException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.errors; /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaBuilderException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaBuilderException.java index 6f0e551625e14..41843c3af8421 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaBuilderException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaBuilderException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.errors; public class SchemaBuilderException extends DataException { diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaProjectorException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaProjectorException.java index 3e2a763b46237..e2b840a730c17 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaProjectorException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/SchemaProjectorException.java @@ -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. + */ package org.apache.kafka.connect.errors; public class SchemaProjectorException extends DataException { diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java index fd99c93b1ba6d..5c5886163e23d 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.sink; import org.apache.kafka.connect.connector.Connector; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java index 894bf0d29d608..e03a1f1396f07 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.sink; import org.apache.kafka.common.record.TimestampType; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java index 0ea1ecf86b5c0..8abff47ce49d6 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java @@ -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 * @@ -13,7 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.sink; import org.apache.kafka.clients.consumer.OffsetAndMetadata; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java index a9d2c05a7e39f..1e214be4b8c05 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.sink; import org.apache.kafka.common.TopicPartition; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java index b67245bcb77d5..0ca3b335fc289 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java @@ -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 * @@ -13,7 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.source; import org.apache.kafka.connect.connector.Connector; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java index 66d7e89307db9..2f3e5e471c934 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.source; import org.apache.kafka.connect.connector.ConnectRecord; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java index 9caca05191176..f3e636bb87af5 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.source; import org.apache.kafka.connect.connector.Task; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java index 674f33c741dd4..8eec1dfb1386f 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java @@ -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 * @@ -13,7 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.source; import org.apache.kafka.connect.storage.OffsetStorageReader; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java b/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java index 763ce5ee09390..507489345b9e9 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/storage/Converter.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.data.Schema; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java b/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java index 10e099cab4a1b..7f94b888965f6 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import java.util.Collection; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverter.java b/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverter.java index 5859f18714145..85fef841b0a3d 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverter.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverter.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.common.errors.SerializationException; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java index b17119ffff316..0803fa880dc57 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/transforms/Transformation.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.Configurable; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java b/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java index 6b3eb935e1c5b..7c09093a31626 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import java.util.ArrayList; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java index 0517b663a166e..85a6cfae3f1de 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.connector; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java index f5c6e2fc98054..339ef23ca54e3 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java index 8d6bd5a58cd7d..2ad6574e3f45e 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/DecimalTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/DecimalTest.java index 27f570af4f0a8..9b2cecf126cf2 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/DecimalTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/DecimalTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.junit.Test; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java b/connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java index ff2e24f6a2b01..bc6fe3ece2aa6 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/FakeSchema.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import java.util.List; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/FieldTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/FieldTest.java index e7b3a9db73305..a450ed9dae046 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/FieldTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/FieldTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.junit.Test; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java index 37cc47c73d8ec..f0c5342f77d0f 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.SchemaBuilderException; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java index 101be04322676..151114e2d2937 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java @@ -1,18 +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 org.apache.kafka.connect.data; import org.apache.kafka.connect.data.Schema.Type; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java index 82f6d89a90f68..42345b1986cdf 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java index 45bdc4e14a021..30be388c6f058 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java index 6121160518b4c..9b47254afc938 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java b/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java index 017b2d3f32d79..e860fbb38a26d 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.data.Schema; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java b/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java index 9f8338d3fac02..ea53084b18103 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import org.junit.Test; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java index d42331310a252..449b9b1768c90 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java index 09d4ed8204b5b..706347c196446 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import org.apache.kafka.clients.consumer.OffsetAndMetadata; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java index 4fb33b73ceabf..bf79b8af1be56 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java index 55bd0f916b4fd..b8b6cfdf1335d 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import java.io.BufferedReader; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java index 5ed03f46e8bbc..660a44cd81b4c 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import org.apache.kafka.common.TopicPartition; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java index 8a21c74bf5a50..c7ec9dac572c2 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import org.apache.kafka.clients.consumer.OffsetAndMetadata; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java index 80ff7f5a2785d..f1fb4efeacb16 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import org.apache.kafka.connect.connector.ConnectorContext; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java index 36893139c483c..e8637f2aabed8 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.file; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index c5d4858e4da61..d9b4f32a8740f 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.json; import com.fasterxml.jackson.databind.JsonNode; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java index a918eb5ebc846..8f2171bc4bce7 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.json; import com.fasterxml.jackson.databind.JsonNode; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSchema.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSchema.java index 9005fdabadbd6..e15d97d64b9d1 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSchema.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSchema.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.json; import com.fasterxml.jackson.databind.JsonNode; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java index 77bcfcaa599d5..438daa17e6b1d 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.json; import com.fasterxml.jackson.databind.JsonNode; diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index 74aad332499a4..62b52a0237195 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.json; import com.fasterxml.jackson.databind.JsonNode; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index f9cf2071aa516..fb3d69359e4a6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.cli; import org.apache.kafka.common.utils.Exit; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java index 6cf04c253bc94..0465048c5d165 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.cli; import org.apache.kafka.common.utils.Exit; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index cbcf14c331956..744c3646c8811 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.common.config.Config; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java index d00b81f60a0c5..00a050a310f19 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; public abstract class AbstractStatus { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java index 219baf0876e8f..846ed1a883570 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.runtime.rest.RestServer; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index 7086da2fdbdf6..74aef627d710c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.common.config.AbstractConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorFactory.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorFactory.java index b432071bb73be..fd0d982850745 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorFactory.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorFactory.java @@ -1,22 +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 org.apache.kafka.connect.runtime; import org.apache.kafka.common.utils.Utils; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java index de5d3936ac924..47f2bf4f7ad64 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java @@ -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 * @@ -13,7 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java index b86d6cbe8a419..93fc6f01ca5aa 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderConnectorContext.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderConnectorContext.java index bd933f1fc0501..04c7ad194b03d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderConnectorContext.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderConnectorContext.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.connector.ConnectorContext; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java index 847f527c43dd9..be55daf571513 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.connector.Connector; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java index 7de3b02230021..21abdd0177b7a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java index 27b0408648ddc..651ac74af5d55 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java index acc2d0d6152bc..f93d7d2b31f8a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java index b59b3bbbd4793..eb25b3d0698b1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; /** diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskConfig.java index 9552fe9e2eefd..649bc00a90ebb 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskConfig.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.common.config.AbstractConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java index 173a694e4f6b8..5ee90415b9660 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.util.ConnectorTaskId; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java index b5a7dc4e46ab5..e1d8b1f2e856a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.connector.ConnectRecord; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 3703ed958e591..1801e1b171cc2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.producer.KafkaProducer; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index 249d41a4ae5de..680edaf97dcb4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.common.config.AbstractConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java index b96976dbb77a2..193742974432d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.connector.Connector; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 6de97b2b79f96..d5f337d4b1c61 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java index ede76c474a227..64c8fff52d670 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java @@ -1,14 +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. - **/ - +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.consumer.KafkaConsumer; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index e86924bdb653f..ed15b85e5152e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.producer.Callback; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java index ac7d0778c6f28..8f60e57e005a7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.source.SourceTaskContext; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java index 04fb333f84a1c..43d45d892970a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.util.ConnectorTaskId; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java index ea5ba82d4ba08..cac71ddc73b02 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.connect.runtime.TargetState; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocol.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocol.java index da5286cb5f9a8..dbb33bcbeb26c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocol.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocol.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.common.protocol.types.ArrayOf; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index c7e585cafeb2f..2d77928e118e4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 1cc97897c40e2..a76fd0333c0f1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java index a4211cc231799..ee0270e25def1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime.distributed; /** diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java index 9340eda433d54..5ccea8a42db52 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; /** diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java index 80c08a3d42c7f..922fabe722e8c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java index 42a5f5dd6838d..3c03e7b85565b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java index 2d6af946dfb29..0dae54b227c59 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java @@ -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 * @@ -13,7 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.clients.consumer.internals.AbstractCoordinator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index f61eaa287acc2..88c9e35845641 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.clients.ClientUtils; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerRebalanceListener.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerRebalanceListener.java index bc833e9e76343..6ff5ce48353a4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerRebalanceListener.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerRebalanceListener.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.connect.util.ConnectorTaskId; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index a878fb0eab810..ce62b7ea0b6a4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest; import com.fasterxml.jackson.core.type.TypeReference; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java index 6040563155e4f..49a2f6f682de9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java index 3e73983fe02cb..d5970b59abed5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java index ead24c57c5627..728ecc5e11de2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java index a6ae006907396..abdcf9349eb0a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java index 3faff6582d94e..9179d3b80d893 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java index 097142e945f00..47e05bc6bcc79 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java @@ -1,14 +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. - **/ - +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java index 179c0dbcff6a6..defe2bb2902f1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java index c8e0f6f9ce5dc..1c52d8db99706 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/CreateConnectorRequest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ErrorMessage.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ErrorMessage.java index e9381d9716055..ecc4de56cd4bb 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ErrorMessage.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ErrorMessage.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ServerInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ServerInfo.java index 1d5e8ba20263b..25ce7311bac38 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ServerInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ServerInfo.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java index 3d443a50c50b9..8e6f3d7baac26 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.entities; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/BadRequestException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/BadRequestException.java index 5fcb226aa2b98..bc9c7f273bf10 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/BadRequestException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/BadRequestException.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime.rest.errors; import javax.ws.rs.core.Response; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectExceptionMapper.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectExceptionMapper.java index 67c38e7f91dd6..d3be2e088756b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectExceptionMapper.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectExceptionMapper.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.errors; import org.apache.kafka.connect.errors.AlreadyExistsException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectRestException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectRestException.java index 5dcbcf4133569..f45f72ddd8bd3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectRestException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectRestException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.errors; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java index 0d208022c3cb7..0d39e81e3b679 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.resources; import org.apache.kafka.connect.runtime.ConnectorConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index 366aac5bc432d..5bd70d6229184 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.resources; import com.fasterxml.jackson.core.type.TypeReference; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java index 3364ffd43dfb9..9d94001ecfa04 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.resources; import org.apache.kafka.connect.runtime.rest.entities.ServerInfo; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java index 5637e05a9c793..f950edf53542c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfig.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.standalone; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index ff09c1cde16ea..ca0130e2f9b9a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.standalone; import org.apache.kafka.connect.errors.AlreadyExistsException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java index 6ab5a1b6ba091..b8fd64380cdb7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.runtime.TargetState; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/FileOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/FileOffsetBackingStore.java index f377617260409..32f5a388c246e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/FileOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/FileOffsetBackingStore.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index 03d84ae9dfa48..c30e8400d6c7c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java index 4d66288617a0a..d927de73eb3a9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java index 38a239ea8343a..137d7d11b0bd1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.storage; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java index 781b5bf38b2de..25891f52354f6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.runtime.TargetState; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java index baa8192454a64..ab8130b6ef776 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryStatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryStatusBackingStore.java index f21c4ade4fde0..39d098dd1c59a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryStatusBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryStatusBackingStore.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.runtime.ConnectorStatus; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java index e307eed751b41..9998164ddf5bf 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.runtime.WorkerConfig; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java index b404de2205956..53cc1a3a6f704 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.data.SchemaAndValue; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java index 45b6fadc37ba3..c5d1467c1bf55 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java index 9fdcfc3ee9c4b..cfae221a3045b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.data.ConnectSchema; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java index 69f1fdc78fec7..63294a47525c5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.runtime.ConnectorStatus; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java index 51bb51987dd84..82606b4c0e91a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkConnector.java index 67fca66566f25..2550e516fc696 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkConnector.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.Config; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkTask.java index b0de58d938d1b..0f8e0aca268a5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSinkTask.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.tools; import org.apache.kafka.clients.consumer.OffsetAndMetadata; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceConnector.java index d69e35559f64a..90868d8282cc3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceConnector.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.Config; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceTask.java index d7288f85d2f0e..5053f3bfdac6d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockSourceTask.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.utils.AppInfoParser; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceConnector.java index 249ed71794f47..eeee0f6ca8d32 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceConnector.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceTask.java index 23f8d2f740e8c..2955fb5377457 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/SchemaSourceTask.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.tools; import org.apache.kafka.connect.data.Schema; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index 6746042baa743..5616613ddfd0a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java index c2515a07a4f8a..12bcb353cd875 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkTask.java index 233345253c800..ee58213cb5e9d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkTask.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.tools; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java index b18db6eeb2d0d..cd7e3105457cc 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java index 04362655fde5f..dfd8bac5aa06d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.tools; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/Callback.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/Callback.java index 0db2ce2ffb644..277863b7f74dd 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/Callback.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/Callback.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; /** diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java index 9bb1468681b76..dabb3470950d4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.util; import org.apache.kafka.common.record.InvalidRecordException; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectorTaskId.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectorTaskId.java index 8176d82322fca..b62f87cc343c8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectorTaskId.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectorTaskId.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java index 88bc9a12e582c..d5abed9385cc0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConvertingFutureCallback.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import java.util.concurrent.CountDownLatch; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/FutureCallback.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/FutureCallback.java index 5b0522a764f92..a1519263139b9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/FutureCallback.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/FutureCallback.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; public class FutureCallback extends ConvertingFutureCallback { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index dd591cbe7f304..df8eefc1760b6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import org.apache.kafka.clients.consumer.Consumer; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java index fc3a0ddbfe658..61d0e35c6288c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java @@ -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 - *

    - * 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 org.apache.kafka.connect.util; import org.reflections.vfs.Vfs; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ShutdownableThread.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ShutdownableThread.java index 01dac9034eb29..daf005b2d4904 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ShutdownableThread.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ShutdownableThread.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import org.slf4j.Logger; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java index cb1ef48e5fd41..777717489c0bc 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.util; public final class SinkUtils { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/Table.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/Table.java index f36d3e545abc7..bf7ca15761236 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/Table.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/Table.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.util; import java.util.Collections; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index d7a2b2ac59895..b9276af3cf040 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java index 9c77b363fc1b6..375b9c0512f7b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java index d0fa1c878bad6..99f4dedd29639 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java index a5f740908334a..9e7719808f0af 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.connector.Connector; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index e7779029ee053..47ce1fc1b1a33 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 9739c99e10e95..fb7cf4fb54113 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 789d2c751c347..31204f01b2e0e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.producer.KafkaProducer; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java index 36803db2bca4c..6c2fc4ddded66 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.sink.SinkTask; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 4e90b45012748..47dfcef33f18c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.producer.KafkaProducer; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 6a36957ced916..70d0736a1064d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index 680f4ca82f135..045b5bbcee096 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -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 * @@ -13,7 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.clients.Metadata; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 64d5b5efc072c..8c533721b762f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest; import org.apache.kafka.connect.runtime.Herder; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index b974559ae0499..e033b6b6123ca 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.resources; import com.fasterxml.jackson.core.type.TypeReference; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index 1c8bd380d6486..0b20271b8f7c2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.rest.resources; import com.fasterxml.jackson.core.type.TypeReference; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index e05aa41299b92..04d14b588d5bd 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.runtime.standalone; import org.apache.kafka.common.config.Config; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java index 6055d9ab83521..c6b61b4d2045c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - S*

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index 4a101c3686a8d..9d0f23ede07b5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java index 4a244f0a6b669..f22b328207023 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java index 0e20b6aef3f32..e2f5a406c195e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.storage; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/MemoryStatusBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/MemoryStatusBackingStoreTest.java index 40aee370f70e9..a31915a3fa2c7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/MemoryStatusBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/MemoryStatusBackingStoreTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.runtime.ConnectorStatus; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java index 3f7082fc8764b..5a671ca6a733d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java index bcfcc23db6716..a6a155fc4b7ad 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java index 4c9d920c358d2..a837e66881091 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/MockTime.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/MockTime.java index e13ddb08042c4..554c5f982fad6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/MockTime.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/MockTime.java @@ -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 * @@ -13,7 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.util; import org.apache.kafka.common.utils.Time; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ShutdownableThreadTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ShutdownableThreadTest.java index 5c3c2244a22be..a72937d8ab4eb 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ShutdownableThreadTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ShutdownableThreadTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import org.junit.Test; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TableTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TableTest.java index ee266b59521ad..6d41a9df6ffa5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TableTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TableTest.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.connect.util; import org.junit.Test; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestBackgroundThreadExceptionHandler.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestBackgroundThreadExceptionHandler.java index 12bac988c02b0..8726d5c87f980 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestBackgroundThreadExceptionHandler.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestBackgroundThreadExceptionHandler.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; /** diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java index 3683f9149c573..0883040a33f21 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import java.util.concurrent.CountDownLatch; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ThreadedTest.java index 0241ea3ef9351..dd367dd0526a8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ThreadedTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.util; import org.junit.After; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java index b7063136e9df1..36373eb0e98eb 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java index 1f2ed7c8e626f..1035e97a41433 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.cache.Cache; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java index f32d6ed2e9465..441129a8f9efa 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.cache.Cache; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java index d7ef2aa8429ad..f9946a025721f 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java index f16560e3c5748..74a19cdb82dfc 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java index 6faf84250af07..8bdc15333a8bf 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.cache.Cache; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java index f3076b445719b..d34a43060ab5d 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java index f917a8d65f09a..938ef0fc27729 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java index 504da541f6574..a9d960103a0b8 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.common.cache.Cache; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java index 1abbbc8aaf081..390812f617079 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms.util; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java index 9713b27446f47..d451f008cf666 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms.util; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java index b004f8ae91500..1d2df45240a50 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms.util; import org.apache.kafka.connect.connector.ConnectRecord; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java index da261e79b22de..e2b7c98686a04 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms.util; import org.apache.kafka.connect.data.Schema; diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java index 6ed1e14cb74e7..7629922589ef2 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms.util; import org.apache.kafka.common.config.AbstractConfig; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java index d72179559a50a..b54a908e4ac3d 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.data.Schema; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java index b5f9d93d935e4..299aab3b5e619 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.data.Schema; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java index 07eb003078922..4ce6ad421cba0 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.data.Schema; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java index c96058afbe6de..9211a462103fc 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.data.Decimal; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java index c599265d57200..cc001f11af08e 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.sink.SinkRecord; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java index 9f9d4b7031b52..e3d9d3a60b9be 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.data.Schema; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java index 2aa790f0a2d82..b2b14db8567ab 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.data.SchemaBuilder; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java index cbd95a6eebaaf..595a71c351692 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.source.SourceRecord; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java index e5328d36baebb..69fb0260c4cdf 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.connect.transforms; import org.apache.kafka.connect.data.Schema; diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index 384be74703050..5a5190dd250be 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.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 @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.consumer import kafka.common.{OffsetAndMetadata, TopicAndPartition} diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index 4de2a4c768afd..21101aae50d5a 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -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 * @@ -13,8 +13,7 @@ * 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.javaapi.consumer; import kafka.common.OffsetAndMetadata; diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java index 288ebd9b03ca1..9c899ede0e3fe 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java @@ -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.javaapi.consumer; diff --git a/core/src/main/scala/kafka/metrics/KafkaTimer.scala b/core/src/main/scala/kafka/metrics/KafkaTimer.scala index f89a14fca0a08..7f76f2d764c08 100644 --- a/core/src/main/scala/kafka/metrics/KafkaTimer.scala +++ b/core/src/main/scala/kafka/metrics/KafkaTimer.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.metrics import com.yammer.metrics.core.Timer diff --git a/core/src/main/scala/kafka/server/LeaderElector.scala b/core/src/main/scala/kafka/server/LeaderElector.scala index 14b3fa4be8e65..2433828551412 100644 --- a/core/src/main/scala/kafka/server/LeaderElector.scala +++ b/core/src/main/scala/kafka/server/LeaderElector.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 * diff --git a/core/src/main/scala/kafka/tools/StreamsResetter.java b/core/src/main/scala/kafka/tools/StreamsResetter.java index a79cd39baefee..a61c092bf8f95 100644 --- a/core/src/main/scala/kafka/tools/StreamsResetter.java +++ b/core/src/main/scala/kafka/tools/StreamsResetter.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/examples/src/main/java/kafka/examples/Consumer.java b/examples/src/main/java/kafka/examples/Consumer.java index f9a9c0f5fb425..be062b309df03 100644 --- a/examples/src/main/java/kafka/examples/Consumer.java +++ b/examples/src/main/java/kafka/examples/Consumer.java @@ -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 * diff --git a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java index c1541bfc6276b..f42ed6f4abd1e 100644 --- a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java +++ b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java @@ -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 * diff --git a/examples/src/main/java/kafka/examples/KafkaProperties.java b/examples/src/main/java/kafka/examples/KafkaProperties.java index 853d6bf535825..cd737cf900e29 100644 --- a/examples/src/main/java/kafka/examples/KafkaProperties.java +++ b/examples/src/main/java/kafka/examples/KafkaProperties.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/examples/src/main/java/kafka/examples/Producer.java b/examples/src/main/java/kafka/examples/Producer.java index ee2a59cd2192c..e7be1a078df7f 100644 --- a/examples/src/main/java/kafka/examples/Producer.java +++ b/examples/src/main/java/kafka/examples/Producer.java @@ -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 * diff --git a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java index 7eef51ebcfdd7..28bdb1638c5b4 100644 --- a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java +++ b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java @@ -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 * diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java index ed96e7b6e5aae..fa0adf8509d46 100644 --- a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java +++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java @@ -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 org.apache.kafka.log4jappender; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java index 71bdd94360c7e..34be2e93c8803 100644 --- a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java @@ -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 * diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java index c35f26adb33f7..87aaca8ccbeae 100644 --- a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java @@ -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 * diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java index 5fcd1f32ba652..b2ef6dcad19de 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.streams.examples.pageview; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java index bb60327066b10..fa82c24130b60 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java @@ -1,19 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ + */ package org.apache.kafka.streams.examples.pageview; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java index 918cd652805f4..4f6257ac74a59 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java @@ -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 org.apache.kafka.streams.examples.pageview; import com.fasterxml.jackson.databind.JsonNode; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 47567223862e8..a194f0f8bff08 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -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 org.apache.kafka.streams.examples.pageview; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 67e8cc50c7dfa..37b2bca064c66 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -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 org.apache.kafka.streams.examples.pageview; import com.fasterxml.jackson.databind.JsonNode; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java index 3c1bd8c049a67..b1764d871e446 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java @@ -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 org.apache.kafka.streams.examples.pipe; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index b1d3c52a0bb56..74f0d5e23cf44 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -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 org.apache.kafka.streams.examples.wordcount; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index f9913cac7ed73..790cab7f48119 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -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 org.apache.kafka.streams.examples.wordcount; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java b/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java index ed3d4884a4b57..f733c25526544 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams; import org.apache.kafka.clients.consumer.Consumer; diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 813b49ef0d351..0cdeb73b55a05 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams; import org.apache.kafka.clients.consumer.KafkaConsumer; diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java index 0c1d2afbcba3c..425e2727def19 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java @@ -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 org.apache.kafka.streams; import java.util.Objects; diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 4af8109a226b3..8c8131850b224 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams; import org.apache.kafka.clients.CommonClientConfigs; diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsMetrics.java b/streams/src/main/java/org/apache/kafka/streams/StreamsMetrics.java index a7bd889862b69..773d0232e5b35 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsMetrics.java @@ -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 org.apache.kafka.streams; import org.apache.kafka.common.Metric; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/InvalidStateStoreException.java b/streams/src/main/java/org/apache/kafka/streams/errors/InvalidStateStoreException.java index ca57be0b49b03..a6ee1f1fb95de 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/InvalidStateStoreException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/InvalidStateStoreException.java @@ -1,16 +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 org.apache.kafka.streams.errors; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LockException.java b/streams/src/main/java/org/apache/kafka/streams/errors/LockException.java index c85fd8424064c..d5cf858e48214 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LockException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LockException.java @@ -1,16 +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 org.apache.kafka.streams.errors; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessorStateException.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessorStateException.java index 061b605517f69..1b659bf7e0bf6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessorStateException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessorStateException.java @@ -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 org.apache.kafka.streams.errors; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/StreamsException.java b/streams/src/main/java/org/apache/kafka/streams/errors/StreamsException.java index b90f11f720711..dc6b1e3640aff 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/StreamsException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/StreamsException.java @@ -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 * @@ -13,8 +13,7 @@ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. - **/ - + */ package org.apache.kafka.streams.errors; import org.apache.kafka.common.KafkaException; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/TaskAssignmentException.java b/streams/src/main/java/org/apache/kafka/streams/errors/TaskAssignmentException.java index 8c06b5b6aa50c..2be43d64a21a3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/TaskAssignmentException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/TaskAssignmentException.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java b/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java index 20d739fa27eb1..c177705a536f9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java @@ -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 org.apache.kafka.streams.errors; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/TopologyBuilderException.java b/streams/src/main/java/org/apache/kafka/streams/errors/TopologyBuilderException.java index 0789460c1fd87..387ab12808a88 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/TopologyBuilderException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/TopologyBuilderException.java @@ -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 org.apache.kafka.streams.errors; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java index 786cfa2b727d1..61218490f9356 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java index 92fdf99434ab2..09212a9a37f20 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java index fa1a5f29be5df..3bd4f850e0f46 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java index bcd16b1321a1f..423791de864b7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index 3869721bdaf9d..2f50dc8e4fde6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java index d767f0be62087..01e4df906de34 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.kstream; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java index 53c0a127fc409..9201813358f79 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 64187e724dc88..46c01dec12727 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index 28326892f6bf3..c361cadcef63c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 9de2644b41a05..807a11c10ccc4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.clients.producer.internals.DefaultPartitioner; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java index e2434e37692fb..98ada013556e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java index f0df2c609516f..a2859458c40d6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java index cf1a1f335b2f2..ac8cd1b6ba09c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java index 5a273e4ee135f..f29b598febbac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java index 943092365dffd..20334458f99b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java index 14af875bebcaa..3801691c2d811 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java index 6d18477977e17..427054e8a94df 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java index f5385aeeba5c2..ba8aba9b22670 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index 14598e41e87bb..cf0366d01864c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java index 94de484c28220..7392cb0e4adc5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java index 63e0cc471aaca..0c1a0c960169d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java index b2008f5ba1f4d..6c6ec50496b2f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java index 01c920e95e4e9..2adc060910b72 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index 0c8b3047d5560..2365f0e441159 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java index 63074c37b1593..aa5157d8a052f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index 74fff47f34a19..d44db534635bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKTableKTableJoinValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKTableKTableJoinValueGetterSupplier.java index fa6d2aa37b518..74c0632d9412b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKTableKTableJoinValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKTableKTableJoinValueGetterSupplier.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java index 90a450ea763b2..3ff7e6a86739d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CacheFlushListener.java index c01ed0fd322bf..8544164a9fcbd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CacheFlushListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CacheFlushListener.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java index 0946b6bd5ef93..a328f5d7b140e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/Change.java @@ -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 org.apache.kafka.streams.kstream.internals; import java.util.Objects; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java index ce9be49a71998..768bfd881a7f8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java index 12e06f8552e4b..961e6ffc803f9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java index 4635fc98c8858..f30ab79df9c83 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ForwardingCacheFlushListener.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java index cef6b67b9c8af..34e23752444ab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java index e8930f4e519ac..1b042d54486d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.kstream.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java index 03fbbced193a5..afbdbf94653a4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java index deb98edf41079..3f2186052371b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorSupplier; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index 082b2e2b5e5b8..67b65a3870132 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.Aggregator; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java index 06083b30b7d0a..317c5bf9e7a85 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java index f5c2fbc5c369d..034942619d842 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java index c038cc84e45d6..f51146182bab2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 7cb1176c31d7e..ab9d22738d484 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapper; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java index 2fd7ef9ad0bb7..eb3189ca94453 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ForeachAction; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java index 099bf07ca6189..2f7409682aae1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.KeyValueMapper; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index f325dcfaa552c..79abbb558ebbe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index 2f4b04d252f1c..1d7078a97a41e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.errors.TopologyBuilderException; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 8d2f1a803548a..b322415beefa3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueJoiner; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java index 7b7f8a73a3e51..ee926bc241a60 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.KeyValueMapper; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 906a3842330a1..bac930d0100ae 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java index c3814a8e0944d..ef9a95ed3e883 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java index 66c0f624db6ed..8897a6cf27221 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapper; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java index adf8e38e640bb..ebaf02772ada3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java index 3dc05131316c4..2e72f6884bc33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ForeachAction; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index fa26a7e7e1d74..90e1e94e0b525 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.Reducer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index 70b2b90095465..166d8a8ff3408 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java index f7fa9f9369dec..93cf4102ad251 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index f689c9503fcdd..8260d2327d72d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 43d284693c120..e2ba512bce81d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.Aggregator; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java index 3eca401a2ce5d..3f4442c0a05c8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index e9e6bfc53caa3..973de0f7379b5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.errors.StreamsException; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java index a434a5f684577..774f2357f5b75 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.Predicate; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index e70054b043962..3f801436dd15d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java index 3ad31cdf33b8a..7fa39d98c4992 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableAbstractJoin.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueJoiner; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java index f8e3f15791ae2..d30177cb20d97 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.KeyValueMapper; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java index 558f4928d9c28..ee7a064579535 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinValueGetter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinValueGetter.java index 30bc4baf778f1..c8c3eb72d48a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinValueGetter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinValueGetter.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index ea625a7880e46..8dc330d4c7838 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueJoiner; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinValueGetter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinValueGetter.java index 57b282c0ac94b..45349c593b89d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinValueGetter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinValueGetter.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 0809d65b58506..90a9f776a6221 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueJoiner; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index 938bd436ac838..05ecf409c2afc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueJoiner; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java index ef60fd45f80d6..90610de15a286 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapper; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java index df032806e5c7c..4be6c5b7e05a4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorSupplier; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index 30b314a9704b2..744cc9c104999 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.errors.StreamsException; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index 24cb82cb4b3c2..09714a7aa8dcd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index 93d9de7eb576b..de8bf2e7bb60f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java index a2034a8dae222..714b8c50dfa37 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorContext; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java index 53ec6ba5e71fb..bb69236f0b4b1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetter.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorContext; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java index 2423bf00ade59..aa28e9ab5213e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableValueGetterSupplier.java @@ -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 org.apache.kafka.streams.kstream.internals; public interface KTableValueGetterSupplier { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java index e143ef2cf03a5..e193e52ec34f5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java index 16c7cdfe90a70..d4757aba124a2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java index 2ea273c0bcb3e..40cc880cb1365 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.kstream.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java index ab805acaa2200..2823cf9d3f659 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java index 4834acf9d86ec..7eb9c563ecdb2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TupleForwarder.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.processor.ProcessorContext; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java index 311169e58e0be..b3f8c3c68628f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedDeserializer.java index d85fb70e71458..1cb8d9c737c8c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedDeserializer.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.config.ConfigException; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java index 8cac7aa2f05e6..ffb3697607947 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.config.ConfigException; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java index ba9873b436371..f4a5e812bc47c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java index 1932e5ec56032..49b3c18bd8bd8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java @@ -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. @@ -6,7 +6,7 @@ * (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, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java index 25efcd6eaf391..5e4da4b049348 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.common.Cluster; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ExtractRecordMetadataTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/processor/ExtractRecordMetadataTimestamp.java index cbe024ee6cf42..59575f5161cbc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ExtractRecordMetadataTimestamp.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ExtractRecordMetadataTimestamp.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/FailOnInvalidTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/processor/FailOnInvalidTimestamp.java index d7f64a297d930..9b90c0872aa35 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/FailOnInvalidTimestamp.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/FailOnInvalidTimestamp.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestamp.java index f24fd158c9be3..228f2ee8101c4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestamp.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestamp.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java index 77418b9bde9df..181e97c24b3dc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.common.Cluster; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index beaace349e085..14332a97fad41 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -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 org.apache.kafka.streams.processor; /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index d854a855a8265..ea50fdc29616b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java index 7976e169f77f5..25bbaab5cb9a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java @@ -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 org.apache.kafka.streams.processor; /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateRestoreCallback.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateRestoreCallback.java index 39decec6622e5..2e896c8a015a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StateRestoreCallback.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateRestoreCallback.java @@ -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 org.apache.kafka.streams.processor; /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java index 9aa0932dc656e..2e1d176b56ea4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor; /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java index e4051ca2a485c..b0ad256a9fe36 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor; import java.util.Map; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java index 0c51c5011877d..1e622e5f0861c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java @@ -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 * diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java index 7fc00d1f5fff1..f4c9ce00814c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.streams.errors.TaskIdFormatException; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java index 0de96ba287e16..51c156a46fe4b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index b4dbf6556e56d..827a1524d8d87 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java index 7718b5cea7f1f..bc96110ee915a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java index e00a102ab9222..555ed105b634c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java index 163c936f00e7d..b30621099c397 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index be145d91919af..c1b1c185b1628 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.Consumer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java index 384eb6d466f6b..2e006a0c74498 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java index 7b02d5b934365..d5e30262cae3e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java index be17008b77fdb..92744ce15fccf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import java.util.Map; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamPartitioner.java index 006f01020b0d4..da48e8f1fef5c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamPartitioner.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStreamPartitioner.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java index bc9d89d047d56..34d0c35359764 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.streams.StreamsConfig; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java index 226fddaf942d3..acb32f7b6c9b2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManager.java index 7670ff08a82f9..b058844b88078 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManager.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 1338387a767e1..f11b200791f37 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index a5504d1b453e8..c5d81f0163d34 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java index b4e15f2e573e9..874565575382a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java index c78ed0913800a..57bb3ac81a6d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicConfig.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicConfig.java index c23f1347fd3a1..522e9f8762e46 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicConfig.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 52536e11d4bf2..821881970befb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.requests.MetadataResponse; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java index 655b8b8551dac..ef7d99040410c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java @@ -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 org.apache.kafka.streams.processor.internals; import java.util.LinkedList; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index 0224cdc3fe320..4a1cffb81d8f3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index f761f1663fe5d..55cddcc7dba18 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.streams.StreamsConfig; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 41c5f7b405ef1..dbda062eeb93f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.metrics.Sensor; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java index 55452ad58d350..aa2010353eb53 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index a06baed0c5b50..840b419ada258 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java index 10042fbbaec1b..8a29786e7123c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.streams.processor.StateStore; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java index 824e20a295f63..0f5185237b33d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -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 org.apache.kafka.streams.processor.internals; import java.util.PriorityQueue; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java index 98919d2716832..be792ba59fa9e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java @@ -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 org.apache.kafka.streams.processor.internals; public class PunctuationSchedule extends Stamped { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java index d99e2ae960415..4bac97da77461 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java @@ -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 org.apache.kafka.streams.processor.internals; public interface Punctuator { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java index 4e789fa6bce1e..b395d42dc94bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java @@ -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 org.apache.kafka.streams.processor.internals; import java.util.HashMap; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java index 730b37dd6c42b..1fa5b9770533b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index f243d82568270..685db54b94b6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.producer.Callback; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordContext.java index f37f3e9f6e958..dc752cb2d9eb0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordContext.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 1129a714f1647..4e04108e39bf7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 077a4d843bae4..0902614ced2e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index f144079416fc4..94cb1f33eeb17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.serialization.Serializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index d44929c2cc7f7..9f46a2cf1ca5b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializer.java index d70af251f0ebf..0e9f1ec8a8cde 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializer.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java index 4e44667397d06..4965238e8568f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java @@ -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 org.apache.kafka.streams.processor.internals; public class Stamped implements Comparable { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java index febd9389fd44c..243c41a8e8bd7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index 82f2714be9cb6..00dbb3c0c6a30 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.TopicPartition; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 712740c59dfb7..09e11fc6173e9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.Consumer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index 2d3e2fc7f2c90..771bb61369cfa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java index 3102b77163086..791892bd1b0f6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java index b2de0f017597b..4edd71c6998f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateRestorer.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index ec9e4543d8e6b..85104fa0133b1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index e17d96b61cbf9..21b9109285318 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 983c07e3f8455..609878a657bf0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.CommitFailedException; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index e5a4562c7d90f..7f482009cde52 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.CommitFailedException; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 362050f4bf2cd..8ee5fe03d9201 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.ClientRequest; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java index 6fb6e062a4c84..67a26bfa30367 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java index bccf736d0a8ad..ec35c8605dbba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.Metric; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java index d8a012a1bdcba..9d56b968d85da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java @@ -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 org.apache.kafka.streams.processor.internals; /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index ddbd67d8d8815..f35de402fead1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.common.record.ByteBufferInputStream; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java index c5577e5ae6638..3e9a521d94cfc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java @@ -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 org.apache.kafka.streams.processor.internals.assignment; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java index 7c81f6438f2cf..f06ecaee6973b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index c3481c0515630..f583dbafc94f1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.streams.errors.TaskAssignmentException; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java index b846ae0ee1624..162add0bd4382 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/HostInfo.java b/streams/src/main/java/org/apache/kafka/streams/state/HostInfo.java index ce75d911ea610..6f485434b8d8e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/HostInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/HostInfo.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java index f9bab2df55edf..3f446351326d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java index cac6b750c289c..a6addba9c3a21 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreType.java b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreType.java index 0a8521b6eab64..6383a9c0fbc18 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreType.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreType.java @@ -1,16 +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 org.apache.kafka.streams.state; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java index a783e07d776ee..e6bc11219919c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/QueryableStoreTypes.java @@ -1,16 +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 org.apache.kafka.streams.state; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java index 880c676069fe8..c638085da831b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java @@ -1,16 +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 org.apache.kafka.streams.state; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index f09d357c59300..68f0719eb68d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 80da9e8f9cfa1..3606c570fd834 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -1,16 +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 org.apache.kafka.streams.state; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java index 20a65f1161b79..1dba933f12502 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java index bb82f6dea2d54..16fa27112da8f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java index b19510c0b0276..43663119c57fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java @@ -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. @@ -6,7 +6,7 @@ * (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, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index 437f7c09a4644..bb5abf8e80e92 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -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. @@ -6,7 +6,7 @@ * (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, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java index 9602bfe10a5ab..198b1b881d1e7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index fb70f42d4362b..c9d5d7166cc70 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state; import org.apache.kafka.common.annotation.InterfaceStability; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java index 86aae77e62908..fdf0936358eec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java index bf0f33fecef2b..438c5b2f05444 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Bytes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractStoreSupplier.java index cde8b36161142..92e8ce08f521f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractStoreSupplier.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachedStateStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachedStateStore.java index a1dcf34e27b14..2f0fa1c04ffaf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachedStateStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachedStateStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 4f4f37b6ce312..a5a5618caba9f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index c33e87ca8fee1..58c013328741c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index 3cc84781e4289..7ea2fa4af3498 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java index 4f3bf480ece2e..f5ad3acd96e40 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java index 022803a6c3da4..c60278f2c0b8e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java index 08493787ab388..3082426801f5d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java index b6b2031af04f1..6366351cbf027 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStore.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java index ea10b1acf12a2..5f4fc64e5ca7e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index 3ffbbb6a754ed..22ec3c6b9606f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java index af1fcdd871bb6..673a7c9ba2780 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java index 4957c0343607e..29f2aeeabd1c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProvider.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/HasNextCondition.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/HasNextCondition.java index 5171ebdae2f5b..8784dbad1bc5b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/HasNextCondition.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/HasNextCondition.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java index c92a1b4f76f3a..bcc98191a97ac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index 28dea79acf226..f63d2f1aefcd7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java index 4be2c4f695b2b..6d1e6dd422009 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java index c2b56fb2ffed4..edbef07af8121 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/LRUCacheEntry.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/LRUCacheEntry.java index e20f40997e461..dedb906f0972d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/LRUCacheEntry.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/LRUCacheEntry.java @@ -1,13 +1,13 @@ -/** +/* * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index c1eb689762c8e..cf78165c4749d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -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. @@ -6,7 +6,7 @@ * (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, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java index 7666155bd4a3f..268042d95d3f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java @@ -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. @@ -6,7 +6,7 @@ * (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, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java index 91673daeed3b1..07d5b6e4de98f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java index db7e898bf8af5..72b73ed9b1c68 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java index a9d097320086c..742bcbca290ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 0a727592a4e23..2f280fc522b81 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Sensor; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java index 3bc1bf6004c54..afc18e468c85e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Sensor; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index 8dcbf904668f3..6ea32a96e0195 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java index 7d10055d4d6b8..57c7a85ce7cf6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.TopicPartition; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/PeekingKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/PeekingKeyValueIterator.java index 496a424505a80..1554ad2892928 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/PeekingKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/PeekingKeyValueIterator.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java index 419fc2831377f..b439ad5037ea3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/QueryableStoreProvider.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java index b72bbede4c52b..82cf96da0dd9a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java index ecd40889c8551..a9d9259b70334 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java index 6c5efcb23f48f..cdd0dd71406d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java index 9eb944738ecbd..6743a7e91abfd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 13d96d3c9a178..db4a03f33fa6e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 8c7a5579aac3d..b82e41605eaf4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java index 9782226cf901a..1b270a2acdf90 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java index 5bbdaabca679e..f3f42a22733b0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java index c449fdbdb118d..9d0b6cdf1664c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentIterator.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index ab08fe6ec319a..622ed08fad44e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java index bc2f4fe4390b6..41cef54b50777 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java index e36e567b59490..2c050f5578aeb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java index 604abb321382e..cd2a4f6409297 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreProvider.java index d8912ddc207f4..bc140916e5cc5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StateStoreProvider.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java index 39c8270600bd6..1e217fa43702b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index 3a50a68a7d09b..2d7ff82241639 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java index d47b636b450ab..528181452b587 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java @@ -1,13 +1,13 @@ -/** +/* * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java index ea607c07dd0a9..7ed598ed97eb7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreSupplier.java index 8d0d6731665fb..ad24c257be47b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreSupplier.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java index d97b0531b27eb..b93e39a190d2f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedSessionStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedSessionStoreIterator.java index b8db3a6fa164e..819f263ad4f36 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedSessionStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedSessionStoreIterator.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java index cbde4e765f6e0..b0721b15a6810 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedWindowStoreIterator.java index 1e48324aebf31..1ce6b0492e7e7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedWindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedWindowStoreIterator.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java index eb1bc6473870a..fdc01c622d7e3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index 5dae8ddf580ad..d4bf471df41c3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java b/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java index 805fa18369e62..ce60fef660137 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index f03bed95c1a35..d345cbda2299a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java index 1d2a3e217e116..8450af5058b70 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.integration; - import kafka.utils.MockTime; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java index 9e59113ada28e..4d9b3658b4ba8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 52bd6e7e00729..3a9d843c1386c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java index ac8abd45a459d..6f777163d8574 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java index f2a767ceb514e..039be448286c8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java @@ -1,12 +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

    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 org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index beb41ce68c610..2551717f23fe3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java index 0a16494397c71..036a3ee0665e4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java index f08bc72f83429..46084a9e5e9e8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java index 2171de1e1e075..c12b975130fe4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java @@ -1,19 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - - package org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java index 7061507f6e688..baeaf6fcc9854 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index 5e8b69e02bffd..fbb708931708d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.streams.integration; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java index 213fffe561332..e115861ad755b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java @@ -1,19 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - - package org.apache.kafka.streams.integration; import kafka.utils.MockTime; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java index f1a8f6844e06c..3248b2ad95efb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. @@ -57,7 +57,6 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; - /** * Tests local state store and global application cleanup. */ diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index 656959a89aad0..25dd0afcaef63 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.integration.utils; import kafka.server.KafkaConfig$; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java index 2680a31b30f54..1ca6bd4dab71e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.integration.utils; import kafka.api.PartitionStateInfo; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java index 9b48272343e4a..18c199588ca52 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.integration.utils; import kafka.admin.AdminUtils; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java index 24387ad687805..0611704ee4ec1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java index a469f25fbe754..3c66dc996a803 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -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 org.apache.kafka.streams.kstream; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java index a9eced46cf1a3..803080b812545 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java index 6b8b6ea9c99c1..7e6bb3e55804b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.streams.kstream.internals.TimeWindow; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java index ea9078c9dbaca..e1f530b7c7a93 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.kstream; import org.apache.kafka.streams.kstream.internals.UnlimitedWindow; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java index 55c5c60d430a9..e7a579e5621ba 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.kstream; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java index 890265f917776..77faf1a1fe887 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java index c41d9539f597e..2526154d17cb6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 3fd287d7304b4..f4cefe422c93f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java index 5fded02dec1bb..8888bfb6050dd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index 7e828ae3aeda2..f9f09eac1770b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index 347b05b620604..7faf6d57f8022 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index 41e0c5bac2e6e..29ea685435f8e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index f4dd08a44cf85..a51c6a466b1d8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java index 0597dc189eed0..ab56ad95ceec1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 9aa2cc8468421..23b03892bab04 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index e11da4c59dd91..aad2351bcc80d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index b3ba4b66c92ee..f9189ea7025ae 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java index 94f68ddfd7e79..662b6e279aaa5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index b6988e31587e6..0f790022ba696 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index f6d1b8f3b2652..b55a60aa74381 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index dd061993ec5cd..acef6e433977b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java index 48f4b653b8a53..7991a923a78f5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPeekTest.java @@ -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,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -import java.util.ArrayList; -import java.util.List; - import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.ForeachAction; @@ -32,6 +25,12 @@ import org.junit.After; import org.junit.Test; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + public class KStreamPeekTest { private final String topicName = "topic"; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java index 94daa3fe352c9..fe6f98bd2c6c7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java @@ -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,11 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.kafka.streams.kstream.internals; - import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.ForeachAction; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index 2e5b201186199..77220e5f3a299 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index 2d4a5305a8a46..6db1d8eed353a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index 0fb082312ac91..0a03e6baa1bde 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 960ef1618a232..856cf73f05637 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index 68700cb4928d5..23325c499699f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 9746d7c93febf..3ab430042ae65 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java index f2a16eea96176..d47b1e661fff8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 37ebdb9d5dca6..7aef28a65535a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java index 83cf96518f5d5..e3d3e95305864 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index 9483c6ba3ba1c..dce19e38dc122 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 65e5595de72dd..a60eae3d1a873 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java index 864b27421e783..5ff01e6ab9fb5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java @@ -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 org.apache.kafka.streams.kstream.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 096f1c1d434c0..3b590c6d03cc6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 5602555f9a5ea..20a6e9a7a831d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java index 4ab1c35d0b2a9..6c0162c9dd18a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java index 3be6373441178..65c4d9fa37b98 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowTest.java index 2df7741213f6a..1bcb7ca764541 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowTest.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.kstream.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java index efa20b8be7464..09ac173b737e3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.kstream.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java index f3c9cfbff89c9..a5c21ea7e4e6c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.kstream.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java index 0b6288fb4d4dd..3af50d8c4de30 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java @@ -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 org.apache.kafka.streams.kstream.internals; import org.apache.kafka.clients.producer.internals.DefaultPartitioner; diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index 4f41e1746d620..dc1bdf52ebc5c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -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 org.apache.kafka.streams.perf; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java index e26453dc15846..de197fe3573d0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.common.Cluster; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/FailOnInvalidTimestampTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/FailOnInvalidTimestampTest.java index 738e956cddc1b..d5607cf74138e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/FailOnInvalidTimestampTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/FailOnInvalidTimestampTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java index 92d87094522b8..93ea448fbdc43 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java index 93e0b5bffa576..8fd955602e948 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java index 23d6704d0387f..c3474b0934482 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -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 org.apache.kafka.streams.processor; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java index 09617fa996990..4fe9fabdf4fbe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java index b7b49bbf53740..12313b75aa134 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/WallclockTimestampExtractorTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java index dee3e965f3de3..6adaa42387b38 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java index dd2ed009fd4c3..58df916439879 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractTaskTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.Consumer; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/CopartitionedTopicsValidatorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/CopartitionedTopicsValidatorTest.java index e7235b03c9bc3..77001ce72f16c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/CopartitionedTopicsValidatorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/CopartitionedTopicsValidatorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.Cluster; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java index 062079fefcbea..8f080512febc1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java index 66999bc8efbe1..4022ba970ff1e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java index cfceaf387c5e1..97372b8e48716 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.MockConsumer; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicConfigTest.java index b0a198bd00844..9c3ef316a1d3a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicConfigTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.utils.Utils; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index 539965390e431..84f0e8a18a681 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.Node; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java index 0a1f95c29e0a8..c398dc523d016 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java @@ -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 org.apache.kafka.streams.processor.internals; import static org.junit.Assert.assertEquals; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java index f6cbbf22ac79a..62661d25a72d4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockStreamsMetrics.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java index ecb9f276981a0..447843f5ab698 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import static org.junit.Assert.assertEquals; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index f2c7e53a0aa1c..b183299142390 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index 31a5d579eb386..a1e1c0191fbba 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index a0b2b8e2b48f0..2f387cba282fe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java index 63b2b750ae32a..a23ff75f59f01 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java index c40e881e0ba75..56e2410419b53 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/QuickUnionTest.java @@ -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 org.apache.kafka.streams.processor.internals; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index f0d06f715d7de..5798514ed631a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.producer.Callback; @@ -42,9 +41,6 @@ import static org.junit.Assert.assertEquals; -/** - * Created by yasuhiro on 2/8/16. - */ public class RecordCollectorTest { private List infos = Arrays.asList( diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordContextStub.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordContextStub.java index 3d5f3e320c634..7932d1ffa44c7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordContextStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordContextStub.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java index a10e8f8fb6ca6..f80b7ebb5429f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import static org.junit.Assert.assertEquals; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java index 0da5974a62ff6..00c77af49abf4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.Cluster; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java index 0c00e6988dcad..718120a6952c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.producer.MockProducer; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java index cf328eee808ca..b2b10160423d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 31aa4a563e1b9..b3a8dab006f7b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java index d818d36c7a424..8ab6052014ad6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateConsumerTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index 7b8afa3a14195..23fed217ba134 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.utils.MockTime; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java index f4aca9fa04aaf..0b6df2faf6575 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerStub.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java index d457887c44ee3..ead19c77a3c18 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateRestorerTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.TopicPartition; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index d6fae66bd7018..b42406f32916f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index 36d652a530793..97d8815c79dcf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 14fda49d6ca88..c1dce593dfbd1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 38797100f4098..87b30b2ea6225 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import java.io.File; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClientTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClientTest.java index 2fb57247e76f1..2b43ebbbb8663 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClientTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClientTest.java @@ -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 - * - * 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 org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.config.AbstractConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index 5bdc4f1797708..c121d96378ba1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java index c6bc2508f2d8c..0e87a6d3858d0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java index cfa0e61dc85ea..9473a4027c456 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java @@ -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 org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.common.TopicPartition; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java index 6a12191cdbefa..669284446213b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.common.utils.Utils; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java index a119d184ca012..a782ea3888228 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java index cf6d8c6560d5b..b71319a01332c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java @@ -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 org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.streams.processor.TaskId; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index 2f5b36851d27c..08adbf46f31c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java index 2e4762694c44e..e7bd1871f0d86 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java index 18d158d56a96a..c2aa14d4f3f46 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index 13f718c301047..05a85ebeb5779 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -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 org.apache.kafka.streams.state.internals; import static org.junit.Assert.assertEquals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java index dc510c56f5c95..ecd662c6409ac 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java index 047503137f927..59caaaf02fa16 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 39947b9db6335..c7b684671b50d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java index 123d8ff7dcd43..4faba5d4048c4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java index 27503ea1a16fa..84f8b2ffbeae0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java index 6ba7b14f43623..00a3b5455d0fe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java index 763ca48f10ff1..e19c4efc4f938 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java index 51b3bf0a4b5e6..e2646adfa1b3a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java index dc1fc5e4a7985..d0c82e46f8aa7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java index 3f5ea6ddc03b0..bc21a7ac02048 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java index ab2ef7e7b8f54..002f7b04fca8e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.streams.errors.InvalidStateStoreException; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java index b44583d8c5cad..52ea9bdef0762 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java @@ -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 * diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java index ed176f27ae80a..2f33c276f8d0b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java @@ -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. @@ -6,7 +6,7 @@ * (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, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIteratorTest.java index 161b7915cea95..3f05428f59475 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIteratorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java index 38e577b635b75..5f24fdea8b1c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedSessionStoreIteratorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java index b8e9b941832a6..5bc4e88f6b8e1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWrappedWindowStoreIteratorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java index 205e990a12aa9..d7cbf11d3fdd1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.Metric; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java index 050e3da88fe70..59297b5174ccd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/NamedCacheTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java index 0d80cd3242bde..3b78d0567d8ae 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/OffsetCheckpointTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import java.io.File; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/QueryableStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/QueryableStoreProviderTest.java index fcf8760fcf384..95258f1188ff8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/QueryableStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/QueryableStoreProviderTest.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 92fbb2c468584..e37e0b452af6c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java index 304772b4ec52f..08ca5e2090d97 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java index b49c01fd6c21f..d11a1e1b3b77f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java @@ -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 * diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java index 3763290765e85..ab13c241dd949 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java index 48618b7e7343e..58db9ad24ea54 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java index e1801b87cc04e..9082da0306225 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java index 98fac08694ad1..d50ecd5b4a42a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index 6286dde8fc91a..b7dd94257f497 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -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 org.apache.kafka.streams.state.internals; import org.apache.kafka.clients.producer.MockProducer; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index 346d3defa3879..0159f0bda24b4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java index 9e34e63062be7..08dcf91b898ed 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java index 8c0d2fec0f5db..8d6cfdae5806f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java @@ -1,20 +1,19 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java index 5fc9e1ffed26b..39cc5b5f77fcf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java index 9b2bd538dbc9a..6edda74b7d41f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java @@ -1,22 +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. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ - package org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index 007cf7fd24d30..3886be82dc3f0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -1,16 +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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java index 7b112e053d090..af7029194c3e1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java @@ -1,13 +1,13 @@ -/** +/* * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowStoreUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowStoreUtilsTest.java index 3a40b44946aeb..372acd9755a06 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowStoreUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowStoreUtilsTest.java @@ -1,20 +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 - *

    - * 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 org.apache.kafka.streams.state.internals; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java index 708e1534dcc1e..76a8747e63d8a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java index 9af32626d5278..6d99e52bddb9b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java @@ -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 * diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java index 5017cded58b94..661dcba738f85 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index 35f1883b484d6..5e6a8b38fe453 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -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 org.apache.kafka.streams.tests; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java index 0d905fbe96e98..b3520fb7e63d1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -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 org.apache.kafka.streams.tests; import org.apache.kafka.clients.consumer.ConsumerConfig; diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index 73fe27c465998..309c90d715ad5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -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 org.apache.kafka.streams.tests; import org.apache.kafka.common.serialization.Serde; diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java index 304cae7e0ada8..d24ad4fccf72d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -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 org.apache.kafka.streams.tests; import java.io.File; diff --git a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java index 612a0da47e6d4..07d588e2144ee 100644 --- a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java +++ b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index dfa2987d04855..f79da31755bc7 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -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 org.apache.kafka.test; import org.apache.kafka.common.metrics.Metrics; diff --git a/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java b/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java index ffcae7e52b72e..0efb8d92585a7 100644 --- a/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java +++ b/streams/src/test/java/org/apache/kafka/test/KTableValueGetterStub.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/KeyValueIteratorStub.java b/streams/src/test/java/org/apache/kafka/test/KeyValueIteratorStub.java index 46423070c3488..2ee5a6fcf9e22 100644 --- a/streams/src/test/java/org/apache/kafka/test/KeyValueIteratorStub.java +++ b/streams/src/test/java/org/apache/kafka/test/KeyValueIteratorStub.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/MockAggregator.java b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java index d8890f6cc5640..eabb31552c924 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockAggregator.java +++ b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java @@ -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 org.apache.kafka.test; import org.apache.kafka.streams.kstream.Aggregator; diff --git a/streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java b/streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java index afa87a896ae4d..4db10e7c39f59 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java +++ b/streams/src/test/java/org/apache/kafka/test/MockChangelogReader.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java index 3861ff854dd01..c867ad7828b7c 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockClientSupplier.java @@ -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 org.apache.kafka.test; import java.util.Map; diff --git a/streams/src/test/java/org/apache/kafka/test/MockInitializer.java b/streams/src/test/java/org/apache/kafka/test/MockInitializer.java index 9bfe7f88eaeeb..61987e7ef129c 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockInitializer.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInitializer.java @@ -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 org.apache.kafka.test; import org.apache.kafka.streams.kstream.Initializer; diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java index 28aca34b23bbe..58c6c6dccdd18 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java @@ -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 org.apache.kafka.test; import org.apache.kafka.clients.consumer.MockConsumer; diff --git a/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java index 870a397001490..2ad24d7592eef 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java +++ b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java @@ -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 org.apache.kafka.test; import org.apache.kafka.streams.KeyValue; diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 93f0f42370306..f4ff09421a892 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -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 org.apache.kafka.test; import java.io.File; diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java index 3ba3603b7cdd1..2fe44f092e5a9 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java @@ -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 org.apache.kafka.test; import org.apache.kafka.streams.processor.ProcessorContext; diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java index a0ffd49ad2b47..571e084606795 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java @@ -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 org.apache.kafka.test; import org.apache.kafka.streams.processor.AbstractProcessor; diff --git a/streams/src/test/java/org/apache/kafka/test/MockReducer.java b/streams/src/test/java/org/apache/kafka/test/MockReducer.java index fc71adae83bde..4db05d3ba096f 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockReducer.java +++ b/streams/src/test/java/org/apache/kafka/test/MockReducer.java @@ -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 org.apache.kafka.test; import org.apache.kafka.streams.kstream.Reducer; diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java index f61048eb5d43b..a69c1504a973f 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java +++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreCallback.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java index 2178c16949143..ce889170bbffa 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java +++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java index f4f84fda317c1..8df4e6cdede12 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -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 org.apache.kafka.test; diff --git a/streams/src/test/java/org/apache/kafka/test/MockStateStoreSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockStateStoreSupplier.java index d1fe213965d37..5d6a4fa28c5ab 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockStateStoreSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockStateStoreSupplier.java @@ -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 org.apache.kafka.test; import org.apache.kafka.common.serialization.Deserializer; diff --git a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java index 2b245784f9cba..17011643797ee 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java @@ -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 org.apache.kafka.test; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java index 4d2dd6925db49..9005ee0b58dac 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java +++ b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java @@ -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 org.apache.kafka.test; import org.apache.kafka.streams.kstream.ValueJoiner; diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java index 2f948780ec326..8e399c5c565e3 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java b/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java index b7fc981ecbc44..0ada2e4432d57 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpReadOnlyStore.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java b/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java index c2833ad942642..3349ae1eb882c 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index 842f30cd3edb3..d2fb1c7e8f218 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java index 61694747014c6..f7fd53ba69703 100644 --- a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java b/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java index 3092d47bab134..6ab70ae0b7f44 100644 --- a/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/test/SegmentedBytesStoreStub.java @@ -1,13 +1,13 @@ -/** +/* * 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 - *

    - * http://www.apache.org/licenses/LICENSE-2.0 - *

    + * 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. diff --git a/streams/src/test/java/org/apache/kafka/test/StateStoreProviderStub.java b/streams/src/test/java/org/apache/kafka/test/StateStoreProviderStub.java index f17777fab3e5b..75340713f5a92 100644 --- a/streams/src/test/java/org/apache/kafka/test/StateStoreProviderStub.java +++ b/streams/src/test/java/org/apache/kafka/test/StateStoreProviderStub.java @@ -1,16 +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 org.apache.kafka.test; diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index a2f3efee63201..6b5be8997edab 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -1,16 +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 - *

    - * 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 org.apache.kafka.test; diff --git a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java index 663fa4a66be82..9efb2dd5d6a9b 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java +++ b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java @@ -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 * @@ -16,8 +16,6 @@ */ package org.apache.kafka.tools; -import static net.sourceforge.argparse4j.impl.Arguments.store; - import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParserException; @@ -47,15 +45,17 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.List; -import java.util.HashMap; import java.util.Map; import java.util.NoSuchElementException; import java.util.Properties; import java.util.concurrent.Future; +import static net.sourceforge.argparse4j.impl.Arguments.store; + /** * ClientCompatibilityTest is invoked by the ducktape test client_compatibility_features_test.py to validate * client behavior when various broker versions are in use. It runs various client operations and tests whether they diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index 411709039158f..0d0902b958715 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -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 org.apache.kafka.tools; diff --git a/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java b/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java index 37ced2d56a639..cfcd04daf79c3 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java +++ b/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java @@ -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 org.apache.kafka.tools; diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java index 0ffe26f948fe7..8bd6a451d62ec 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java @@ -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 * @@ -58,7 +58,6 @@ import static net.sourceforge.argparse4j.impl.Arguments.store; import static net.sourceforge.argparse4j.impl.Arguments.storeTrue; - /** * Command line consumer designed for system testing. It outputs consumer events to STDOUT as JSON * formatted objects. The "name" field in each JSON event identifies the event type. The following diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java index 14d16426e0a78..9df5f613eb591 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java @@ -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 * @@ -16,7 +16,6 @@ */ package org.apache.kafka.tools; - import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParserException; diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java index 367d9c2b645b4..f8f72333301ac 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java @@ -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 org.apache.kafka.tools; import org.apache.kafka.clients.producer.Callback; From e31c0c9bdbad432bc21b583bd3c084f05323f642 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 28 Feb 2017 13:30:10 -0800 Subject: [PATCH 045/101] MINOR: Fix typo in javadoc of `flatMapValues` Author: Michael G. Noll Reviewers: Damian Guy, Matthias J. Sax, Guozhang Wang Closes #2607 from miguno/trunk-flatMapValues-docstring --- .../main/java/org/apache/kafka/streams/kstream/KStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 46c01dec12727..a11d8f443b8a7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -192,7 +192,7 @@ public interface KStream { * and emit a record {@code } for each word. *

    {@code
          * KStream inputStream = builder.stream("topic");
    -     * KStream outputStream = inputStream.flatMap(new KeyValueMapper>> {
    +     * KStream outputStream = inputStream.flatMap(new KeyValueMapper>> {
          *     Iterable> apply(byte[] key, String value) {
          *         String[] tokens = value.split(" ");
          *         List> result = new ArrayList<>(tokens.length);
    @@ -238,7 +238,7 @@ public interface KStream {
          * The example below splits input records {@code } containing sentences as values into their words.
          * 
    {@code
          * KStream inputStream = builder.stream("topic");
    -     * KStream outputStream = inputStream.flatMap(new ValueMapper> {
    +     * KStream outputStream = inputStream.flatMapValues(new ValueMapper> {
          *     Iterable apply(String value) {
          *         return Arrays.asList(value.split(" "));
          *     }
    
    From a10d7b1b765a2ac4af3e219b7c344f67a93de487 Mon Sep 17 00:00:00 2001
    From: Hamidreza Afzali 
    Date: Tue, 28 Feb 2017 13:31:32 -0800
    Subject: [PATCH 046/101] KAFKA-4789: Added support to
     ProcessorTopologyTestDriver to forward timestamps to internal topics
    
    This resolves the issue in the ProcessorTopologyTestDriver that the extracted timestamp is not forwarded with the produced record to the internal topics.
    
    JIRA ticket: https://issues.apache.org/jira/browse/KAFKA-4789
    
    The contribution is my original work and I license the work to the project under the project's open source license.
    
    guozhangwang dguy
    
    Author: Hamidreza Afzali 
    
    Reviewers: Damian Guy, Guozhang Wang
    
    Closes #2590 from hrafzali/KAFKA-4789_ProcessorTopologyTestDriver_timestamp
    ---
     .../internals/ProcessorTopologyTest.java      | 46 +++++++++++++++++++
     .../test/ProcessorTopologyTestDriver.java     | 29 ++++++++----
     2 files changed, 67 insertions(+), 8 deletions(-)
    
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
    index 2f387cba282fe..322c17810c206 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
    @@ -46,6 +46,7 @@
     import java.util.Properties;
     
     import static org.hamcrest.CoreMatchers.containsString;
    +import static org.hamcrest.CoreMatchers.equalTo;
     import static org.hamcrest.MatcherAssert.assertThat;
     import static org.junit.Assert.assertEquals;
     import static org.junit.Assert.assertNotNull;
    @@ -245,6 +246,20 @@ public void testDrivingInternalRepartitioningTopology() {
             assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3");
         }
     
    +    @Test
    +    public void testDrivingInternalRepartitioningForwardingTimestampTopology() {
    +        driver = new ProcessorTopologyTestDriver(config, createInternalRepartitioningWithValueTimestampTopology());
    +        driver.process(INPUT_TOPIC_1, "key1", "value1@1000", STRING_SERIALIZER, STRING_SERIALIZER);
    +        driver.process(INPUT_TOPIC_1, "key2", "value2@2000", STRING_SERIALIZER, STRING_SERIALIZER);
    +        driver.process(INPUT_TOPIC_1, "key3", "value3@3000", STRING_SERIALIZER, STRING_SERIALIZER);
    +        assertThat(driver.readOutput(OUTPUT_TOPIC_1, STRING_DESERIALIZER, STRING_DESERIALIZER),
    +                equalTo(new ProducerRecord<>(OUTPUT_TOPIC_1, null, 1000L, "key1", "value1")));
    +        assertThat(driver.readOutput(OUTPUT_TOPIC_1, STRING_DESERIALIZER, STRING_DESERIALIZER),
    +                equalTo(new ProducerRecord<>(OUTPUT_TOPIC_1, null, 2000L, "key2", "value2")));
    +        assertThat(driver.readOutput(OUTPUT_TOPIC_1, STRING_DESERIALIZER, STRING_DESERIALIZER),
    +                equalTo(new ProducerRecord<>(OUTPUT_TOPIC_1, null, 3000L, "key3", "value3")));
    +    }
    +
         @Test
         public void shouldCreateStringWithSourceAndTopics() throws Exception {
             builder.addSource("source", "topic1", "topic2");
    @@ -356,6 +371,15 @@ private TopologyBuilder createInternalRepartitioningTopology() {
                 .addSink("sink1", OUTPUT_TOPIC_1, "source1");
         }
     
    +    private TopologyBuilder createInternalRepartitioningWithValueTimestampTopology() {
    +        return builder.addSource("source", INPUT_TOPIC_1)
    +                .addInternalTopic(THROUGH_TOPIC_1)
    +                .addProcessor("processor", define(new ValueTimestampProcessor()), "source")
    +                .addSink("sink0", THROUGH_TOPIC_1, "processor")
    +                .addSource("source1", THROUGH_TOPIC_1)
    +                .addSink("sink1", OUTPUT_TOPIC_1, "source1");
    +    }
    +
         private TopologyBuilder createSimpleMultiSourceTopology(int partition) {
             return builder.addSource("source-1", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC_1)
                     .addProcessor("processor-1", define(new ForwardingProcessor()), "source-1")
    @@ -382,6 +406,18 @@ public void punctuate(long streamTime) {
             }
         }
     
    +    /**
    +     * A processor that removes custom timestamp information from messages and forwards modified messages to each child.
    +     * A message contains custom timestamp information if the value is in ".*@[0-9]+" format.
    +     */
    +    protected static class ValueTimestampProcessor extends AbstractProcessor {
    +
    +        @Override
    +        public void process(String key, String value) {
    +            context().forward(key, value.split("@")[0]);
    +        }
    +    }
    +
         /**
          * A processor that forwards slightly-modified messages to each child.
          */
    @@ -487,9 +523,19 @@ public Processor get() {
             };
         }
     
    +    /**
    +     * A custom timestamp extractor that extracts the timestamp from the record's value if the value is in ".*@[0-9]+"
    +     * format. Otherwise, it returns the record's timestamp or the default timestamp if the record's timestamp is zero.
    +    */
         public static class CustomTimestampExtractor implements TimestampExtractor {
             @Override
             public long extract(final ConsumerRecord record, final long previousTimestamp) {
    +            if (record.value().toString().matches(".*@[0-9]+"))
    +                return Long.parseLong(record.value().toString().split("@")[1]);
    +
    +            if (record.timestamp() > 0L)
    +                return record.timestamp();
    +
                 return timestamp;
             }
         }
    diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
    index d2fb1c7e8f218..5808e9ad386a6 100644
    --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
    +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
    @@ -233,23 +233,27 @@ public List partitionsFor(String topic) {
         }
     
         /**
    -     * Send an input message with the given key and value on the specified topic to the topology, and then commit the messages.
    +     * Send an input message with the given key, value and timestamp on the specified topic to the topology, and then commit the messages.
          *
          * @param topicName the name of the topic on which the message is to be sent
          * @param key the raw message key
          * @param value the raw message value
    +     * @param timestamp the raw message timestamp
          */
    -    public void process(String topicName, byte[] key, byte[] value) {
    +    private void process(String topicName, byte[] key, byte[] value, long timestamp) {
    +
             TopicPartition tp = partitionsByTopic.get(topicName);
             if (tp != null) {
                 // Add the record ...
                 long offset = offsetsByTopicPartition.get(tp).incrementAndGet();
    -            task.addRecords(tp, records(new ConsumerRecord<>(tp.topic(), tp.partition(), offset, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, key, value)));
    +            task.addRecords(tp, records(new ConsumerRecord<>(tp.topic(), tp.partition(), offset, timestamp, TimestampType.CREATE_TIME, 0L, 0, 0, key, value)));
                 producer.clear();
    +
                 // Process the record ...
                 task.process();
    -            ((InternalProcessorContext) task.context()).setRecordContext(new ProcessorRecordContext(0L, offset, tp.partition(), topicName));
    +            ((InternalProcessorContext) task.context()).setRecordContext(new ProcessorRecordContext(timestamp, offset, tp.partition(), topicName));
                 task.commit();
    +
                 // Capture all the records sent to the producer ...
                 for (ProducerRecord record : producer.history()) {
                     Queue> outputRecords = outputRecordsByTopic.get(record.topic());
    @@ -261,7 +265,7 @@ public void process(String topicName, byte[] key, byte[] value) {
     
                     // Forward back into the topology if the produced record is to an internal topic ...
                     if (internalTopics.contains(record.topic())) {
    -                    process(record.topic(), record.key(), record.value());
    +                    process(record.topic(), record.key(), record.value(), record.timestamp());
                     }
                 }
             } else {
    @@ -270,11 +274,20 @@ public void process(String topicName, byte[] key, byte[] value) {
                     throw new IllegalArgumentException("Unexpected topic: " + topicName);
                 }
                 final long offset = offsetsByTopicPartition.get(global).incrementAndGet();
    -            globalStateTask.update(new ConsumerRecord<>(global.topic(), global.partition(), offset, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, key, value));
    +            globalStateTask.update(new ConsumerRecord<>(global.topic(), global.partition(), offset, timestamp, TimestampType.CREATE_TIME, 0L, 0, 0, key, value));
                 globalStateTask.flushState();
             }
    +    }
     
    -
    +    /**
    +     * Send an input message with the given key and value on the specified topic to the topology.
    +     *
    +     * @param topicName the name of the topic on which the message is to be sent
    +     * @param key the raw message key
    +     * @param value the raw message value
    +     */
    +    public void process(String topicName, byte[] key, byte[] value) {
    +        process(topicName, key, value, 0L);
         }
     
         /**
    @@ -317,7 +330,7 @@ public  ProducerRecord readOutput(String topic, Deserializer keyD
             if (record == null) return null;
             K key = keyDeserializer.deserialize(record.topic(), record.key());
             V value = valueDeserializer.deserialize(record.topic(), record.value());
    -        return new ProducerRecord(record.topic(), record.partition(), key, value);
    +        return new ProducerRecord(record.topic(), record.partition(), record.timestamp(), key, value);
         }
     
         private Iterable> records(ConsumerRecord record) {
    
    From de05c9d3a0d79a555088afe9344b52e002d287f2 Mon Sep 17 00:00:00 2001
    From: Damian Guy 
    Date: Tue, 28 Feb 2017 22:55:46 +0000
    Subject: [PATCH 047/101] MINOR: Add code quality checks (and suppressions) to
     checkstyle.xml
    
    Author: Damian Guy 
    
    Reviewers: Guozhang Wang , Ewen Cheslack-Postava , Jason Gustafson , Ismael Juma 
    
    Closes #2594 from dguy/checkstyle
    ---
     build.gradle                                  |   5 +-
     checkstyle/checkstyle.xml                     |  36 +++
     checkstyle/suppressions.xml                   | 229 ++++++++++++++++++
     .../kafka/common/protocol/types/Struct.java   |   2 +-
     .../SaslClientCallbackHandler.java            |   2 +-
     .../clients/producer/ProducerRecordTest.java  |  12 +-
     .../apache/kafka/streams/StreamsConfig.java   |   6 +-
     .../kstream/internals/SessionKeySerde.java    |   6 +-
     .../streams/processor/TopologyBuilder.java    |   4 +-
     ...StreamAggregationDedupIntegrationTest.java |  12 +-
     .../KStreamAggregationIntegrationTest.java    |  24 +-
     .../internals/StreamsMetadataStateTest.java   |   7 +-
     .../test/ProcessorTopologyTestDriver.java     |   2 +-
     13 files changed, 307 insertions(+), 40 deletions(-)
     create mode 100644 checkstyle/suppressions.xml
    
    diff --git a/build.gradle b/build.gradle
    index 417383deee7ef..fa2ed969a9484 100644
    --- a/build.gradle
    +++ b/build.gradle
    @@ -109,7 +109,8 @@ if (new File('.git').exists()) {
             'gradlew.bat',
             '**/README.md',
             '**/id_rsa',
    -        '**/id_rsa.pub'
    +        '**/id_rsa.pub',
    +        'checkstyle/suppressions.xml'
         ])
       }
     }
    @@ -272,6 +273,8 @@ subprojects {
       checkstyle {
         configFile = new File(rootDir, "checkstyle/checkstyle.xml")
         configProperties = [importControlFile: "$rootDir/checkstyle/import-control.xml"]
    +    // version 7.x requires Java 8
    +    toolVersion = '6.19'
       }
       test.dependsOn('checkstyleMain', 'checkstyleTest')
     
    diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml
    index 51b613dfe0197..9a4a37f4e5bab 100644
    --- a/checkstyle/checkstyle.xml
    +++ b/checkstyle/checkstyle.xml
    @@ -93,5 +93,41 @@
           
           
         
    +
    +    
    +    
    +    
    +      
    +      
    +    
    +    
    +      
    +      
    +    
    +    
    +      
    +      
    +    
    +
    +    
    +      
    +      
    +    
    +    
    +      
    +      
    +    
    +    
    +      
    +      
    +    
    +    
    +      
    +      
    +    
    +  
    +
    +  
    +    
       
     
    diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
    new file mode 100644
    index 0000000000000..9f133075c5285
    --- /dev/null
    +++ b/checkstyle/suppressions.xml
    @@ -0,0 +1,229 @@
    +
    +
    +
    +
    +
    +
    +    
    +    
    +    
    +
    +    
    +
    +    
    +    
    +    
    +    
    +    
    +
    +    
    +    
    +
    +    
    +
    +    
    +    
    +    
    +    
    +    
    +    
    +    
    +    
    +    
    +    
    +    
    +
    +    
    +
    +    
    +    
    +    
    +    
    +    
    +    
    +    
    +    
    +
    +    
    +    
    +
    +    
    +
    +    
    +    
    +
    +    
    +
    +    
    +    
    +    
    +
    +    
    +
    +    
    +
    +    
    +    
    +    
    +    
    +    
    +
    +    
    +
    +    
    +    
    +    
    +    
    +    
    +
    +    
    +    
    +
    +    
    +    
    +    
    +    
    +    
    +
    +    
    +
    +    
    +    
    +
    +    
    +
    +    
    +    
    +    
    +
    +    
    +
    +    
    +    
    +    
    +
    +    
    +    
    +
    +    
    +    
    +    
    +
    +    
    +    
    +
    +    
    +
    +    
    +    
    +
    +    
    +    
    +
    +    
    +    
    +
    +    
    +    
    +
    +    
    +    
    +
    +    
    +    
    +
    +    
    +    
    +
    +    
    +
    +
    diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
    index 1258763d43477..c32aea7c0dcb4 100644
    --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
    +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
    @@ -316,7 +316,7 @@ public int hashCode() {
                 Field f = this.schema.get(i);
                 if (f.type() instanceof ArrayOf) {
                     if (this.get(f) != null) {
    -                    Object[] arrayObject = (Object []) this.get(f);
    +                    Object[] arrayObject = (Object[]) this.get(f);
                         for (Object arrayItem: arrayObject)
                             result = prime * result + arrayItem.hashCode();
                     }
    diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
    index 3391ff3436026..7111bad6054e3 100644
    --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
    +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
    @@ -58,7 +58,7 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
                         nc.setName(nc.getDefaultName());
                 } else if (callback instanceof PasswordCallback) {
                     if (!isKerberos && subject != null && !subject.getPrivateCredentials(String.class).isEmpty()) {
    -                    char [] password = subject.getPrivateCredentials(String.class).iterator().next().toCharArray();
    +                    char[] password = subject.getPrivateCredentials(String.class).iterator().next().toCharArray();
                         ((PasswordCallback) callback).setPassword(password);
                     } else {
                         String errorMessage = "Could not login: the client is being asked for a password, but the Kafka" +
    diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java
    index 7d725fa6aaa86..b5a7a60cfa351 100644
    --- a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java
    +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java
    @@ -26,24 +26,24 @@ public class ProducerRecordTest {
     
         @Test
         public void testEqualsAndHashCode() {
    -        ProducerRecord producerRecord = new ProducerRecord<>("test", 1 , "key", 1);
    +        ProducerRecord producerRecord = new ProducerRecord<>("test", 1, "key", 1);
             assertEquals(producerRecord, producerRecord);
             assertEquals(producerRecord.hashCode(), producerRecord.hashCode());
     
    -        ProducerRecord equalRecord = new ProducerRecord<>("test", 1 , "key", 1);
    +        ProducerRecord equalRecord = new ProducerRecord<>("test", 1, "key", 1);
             assertEquals(producerRecord, equalRecord);
             assertEquals(producerRecord.hashCode(), equalRecord.hashCode());
     
    -        ProducerRecord topicMisMatch = new ProducerRecord<>("test-1", 1 , "key", 1);
    +        ProducerRecord topicMisMatch = new ProducerRecord<>("test-1", 1, "key", 1);
             assertFalse(producerRecord.equals(topicMisMatch));
     
    -        ProducerRecord partitionMismatch = new ProducerRecord<>("test", 2 , "key", 1);
    +        ProducerRecord partitionMismatch = new ProducerRecord<>("test", 2, "key", 1);
             assertFalse(producerRecord.equals(partitionMismatch));
     
    -        ProducerRecord keyMisMatch = new ProducerRecord<>("test", 1 , "key-1", 1);
    +        ProducerRecord keyMisMatch = new ProducerRecord<>("test", 1, "key-1", 1);
             assertFalse(producerRecord.equals(keyMisMatch));
     
    -        ProducerRecord valueMisMatch = new ProducerRecord<>("test", 1 , "key", 2);
    +        ProducerRecord valueMisMatch = new ProducerRecord<>("test", 1, "key", 2);
             assertFalse(producerRecord.equals(valueMisMatch));
     
             ProducerRecord nullFieldsRecord = new ProducerRecord<>("topic", null, null, null, null);
    diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
    index 8c8131850b224..0eb3f7b24818f 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
    @@ -388,8 +388,7 @@ public class StreamsConfig extends AbstractConfig {
         // this is the list of configs for underlying clients
         // that streams prefer different default values
         private static final Map PRODUCER_DEFAULT_OVERRIDES;
    -    static
    -    {
    +    static {
             final Map tempProducerDefaultOverrides = new HashMap<>();
             tempProducerDefaultOverrides.put(ProducerConfig.LINGER_MS_CONFIG, "100");
     
    @@ -397,8 +396,7 @@ public class StreamsConfig extends AbstractConfig {
         }
     
         private static final Map CONSUMER_DEFAULT_OVERRIDES;
    -    static
    -    {
    +    static {
             final Map tempConsumerDefaultOverrides = new HashMap<>();
             tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "1000");
             tempConsumerDefaultOverrides.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
    diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java
    index d4757aba124a2..7eb8300d6b2a0 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java
    @@ -112,15 +112,15 @@ public void close() {
             }
         }
     
    -    public static long extractEnd(final byte [] binaryKey) {
    +    public static long extractEnd(final byte[] binaryKey) {
             return ByteBuffer.wrap(binaryKey).getLong(binaryKey.length - 2 * TIMESTAMP_SIZE);
         }
     
    -    public static long extractStart(final byte [] binaryKey) {
    +    public static long extractStart(final byte[] binaryKey) {
             return ByteBuffer.wrap(binaryKey).getLong(binaryKey.length - TIMESTAMP_SIZE);
         }
     
    -    public static Window extractWindow(final byte [] binaryKey) {
    +    public static Window extractWindow(final byte[] binaryKey) {
             final ByteBuffer buffer = ByteBuffer.wrap(binaryKey);
             final long start = buffer.getLong(binaryKey.length - TIMESTAMP_SIZE);
             final long end = buffer.getLong(binaryKey.length - 2 * TIMESTAMP_SIZE);
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
    index 827a1524d8d87..99f5d651bbbbf 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
    @@ -294,7 +294,7 @@ public String toString() {
          * Enum used to define auto offset reset policy when creating {@link KStream} or {@link KTable}
          */
         public enum AutoOffsetReset {
    -        EARLIEST , LATEST
    +        EARLIEST, LATEST
         }
     
         /**
    @@ -864,7 +864,7 @@ private void connectProcessorAndStateStore(String processorName, String stateSto
             }
         }
     
    -    private Set findSourceTopicsForProcessorParents(String [] parents) {
    +    private Set findSourceTopicsForProcessorParents(String[] parents) {
             final Set sourceTopics = new HashSet<>();
             for (String parent : parents) {
                 NodeFactory nodeFactory = nodeFactories.get(parent);
    diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
    index 039be448286c8..dcaa2221f1b60 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
    @@ -131,8 +131,8 @@ public void shouldReduce() throws Exception {
     
             List> results = receiveMessages(
                 new StringDeserializer(),
    -            new StringDeserializer()
    -            , 5);
    +            new StringDeserializer(),
    +            5);
     
             Collections.sort(results, new Comparator>() {
                 @Override
    @@ -181,8 +181,8 @@ public String apply(Windowed windowedKey, String value) {
     
             List> windowedOutput = receiveMessages(
                 new StringDeserializer(),
    -            new StringDeserializer()
    -            , 10);
    +            new StringDeserializer(),
    +            10);
     
             Comparator>
                 comparator =
    @@ -233,8 +233,8 @@ public String apply(final Windowed windowedKey, final Long value) {
     
             final List> results = receiveMessages(
                 new StringDeserializer(),
    -            new LongDeserializer()
    -            , 5);
    +            new LongDeserializer(),
    +            5);
             Collections.sort(results, new Comparator>() {
                 @Override
                 public int compare(final KeyValue o1, final KeyValue o2) {
    diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
    index 2551717f23fe3..4eb582cd82805 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
    @@ -155,8 +155,8 @@ public void shouldReduce() throws Exception {
     
             final List> results = receiveMessages(
                 new StringDeserializer(),
    -            new StringDeserializer()
    -            , 10);
    +            new StringDeserializer(),
    +            10);
     
             Collections.sort(results, new Comparator>() {
                 @Override
    @@ -209,8 +209,8 @@ public String apply(final Windowed windowedKey, final String value) {
     
             final List> windowedOutput = receiveMessages(
                 new StringDeserializer(),
    -            new StringDeserializer()
    -            , 15);
    +            new StringDeserializer(),
    +            15);
     
             final Comparator>
                 comparator =
    @@ -263,8 +263,8 @@ public void shouldAggregate() throws Exception {
     
             final List> results = receiveMessages(
                 new StringDeserializer(),
    -            new IntegerDeserializer()
    -            , 10);
    +            new IntegerDeserializer(),
    +            10);
     
             Collections.sort(results, new Comparator>() {
                 @Override
    @@ -313,8 +313,8 @@ public String apply(final Windowed windowedKey, final Integer value) {
     
             final List> windowedMessages = receiveMessages(
                 new StringDeserializer(),
    -            new IntegerDeserializer()
    -            , 15);
    +            new IntegerDeserializer(),
    +            15);
     
             final Comparator>
                 comparator =
    @@ -364,8 +364,8 @@ public void shouldCount() throws Exception {
     
             final List> results = receiveMessages(
                 new StringDeserializer(),
    -            new LongDeserializer()
    -            , 10);
    +            new LongDeserializer(),
    +            10);
             Collections.sort(results, new Comparator>() {
                 @Override
                 public int compare(final KeyValue o1, final KeyValue o2) {
    @@ -406,8 +406,8 @@ public String apply(final Windowed windowedKey, final Long value) {
     
             final List> results = receiveMessages(
                 new StringDeserializer(),
    -            new LongDeserializer()
    -            , 10);
    +            new LongDeserializer(),
    +            10);
             Collections.sort(results, new Comparator>() {
                 @Override
                 public int compare(final KeyValue o1, final KeyValue o2) {
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java
    index c121d96378ba1..c8ab6f1d17406 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java
    @@ -62,7 +62,7 @@ public class StreamsMetadataStateTest {
         private TopicPartition topic4P0;
         private List partitionInfos;
         private Cluster cluster;
    -    private final String globalTable = "global-table";;
    +    private final String globalTable = "global-table";
         private StreamPartitioner partitioner;
     
         @Before
    @@ -200,8 +200,9 @@ public void shouldGetInstanceWithKey() throws Exception {
             final StreamsMetadata expected = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"),
                     Collections.singleton(topic3P0));
     
    -        final StreamsMetadata actual = discovery.getMetadataWithKey("table-three", "the-key",
    -                Serdes.String().serializer());
    +        final StreamsMetadata actual = discovery.getMetadataWithKey("table-three",
    +                                                                    "the-key",
    +                                                                    Serdes.String().serializer());
     
             assertEquals(expected, actual);
         }
    diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
    index 5808e9ad386a6..b704aa70e3623 100644
    --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
    +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
    @@ -202,7 +202,7 @@ public List partitionsFor(String topic) {
                 final MockConsumer globalConsumer = createGlobalConsumer();
                 for (final String topicName : globalTopology.sourceTopics()) {
                     List partitionInfos = new ArrayList<>();
    -                partitionInfos.add(new PartitionInfo(topicName , 1, null, null, null));
    +                partitionInfos.add(new PartitionInfo(topicName, 1, null, null, null));
                     globalConsumer.updatePartitions(topicName, partitionInfos);
                     final TopicPartition partition = new TopicPartition(topicName, 1);
                     globalConsumer.updateEndOffsets(Collections.singletonMap(partition, 0L));
    
    From c9872cb2142bbee12e4657af9a84627a1684fddd Mon Sep 17 00:00:00 2001
    From: "Colin P. Mccabe" 
    Date: Tue, 28 Feb 2017 23:04:02 +0000
    Subject: [PATCH 048/101] KAFKA-4773; The Kafka build should run findbugs
    
    Author: Colin P. Mccabe 
    
    Reviewers: Ismael Juma 
    
    Closes #2557 from cmccabe/KAFKA-4773
    ---
     README.md                   | 21 ++++++++++++++++++---
     build.gradle                | 14 ++++++++++++++
     gradle/findbugs-exclude.xml | 28 ++++++++++++++++++++++++++++
     3 files changed, 60 insertions(+), 3 deletions(-)
     create mode 100644 gradle/findbugs-exclude.xml
    
    diff --git a/README.md b/README.md
    index c219718c06e29..9c2413bca6838 100644
    --- a/README.md
    +++ b/README.md
    @@ -133,11 +133,26 @@ Please note for this to work you should create/update `${GRADLE_USER_HOME}/gradl
     ### Determining if any dependencies could be updated ###
         ./gradlew dependencyUpdates
     
    -### Running checkstyle on the java code ###
    +### Running code quality checks ###
    +There are two code quality analysis tools that we regularly run, findbugs and checkstyle.
    +
    +#### Checkstyle
    +Checkstyle enforces a consistent coding style in Kafka.
    +You can run checkstyle using:
    +
         ./gradlew checkstyleMain checkstyleTest
     
    -This will most commonly be useful for automated builds where the full resources of the host running the build and tests
    -may not be dedicated to Kafka's build.
    +The checkstyle warnings will be found in `reports/checkstyle/reports/main.html` and `reports/checkstyle/reports/test.html` files in the
    +subproject build directories. They are also are printed to the console. The build will fail if Checkstyle fails.
    +
    +#### Findbugs
    +Findbugs uses static analysis to look for bugs in the code.
    +You can run findbugs using:
    +
    +    ./gradlew findbugsMain findbugsTest -x test
    +
    +The findbugs warnings will be found in `reports/findbugs/main.html` and `reports/findbugs/test.html` files in the subproject build
    +directories. Currently, findbugs warnings do not cause the build to fail.
     
     ### Common build options ###
     
    diff --git a/build.gradle b/build.gradle
    index fa2ed969a9484..caac99d1e25c9 100644
    --- a/build.gradle
    +++ b/build.gradle
    @@ -125,6 +125,7 @@ subprojects {
       apply plugin: 'maven'
       apply plugin: 'signing'
       apply plugin: 'checkstyle'
    +  apply plugin: 'findbugs'
     
       sourceCompatibility = 1.7
     
    @@ -278,6 +279,19 @@ subprojects {
       }
       test.dependsOn('checkstyleMain', 'checkstyleTest')
     
    +  findbugs {
    +    toolVersion = "3.0.1"
    +    excludeFilter = file("$rootDir/gradle/findbugs-exclude.xml")
    +    ignoreFailures = true
    +  }
    +
    +  tasks.withType(FindBugs) {
    +    reports {
    +      xml.enabled false
    +      html.enabled true
    +    }
    +  }
    +
       // Ignore core since its a scala project
       if (it.path != ':core') {
         // NOTE: Gradles Jacoco plugin does not support "offline instrumentation" this means that classes mocked by PowerMock
    diff --git a/gradle/findbugs-exclude.xml b/gradle/findbugs-exclude.xml
    new file mode 100644
    index 0000000000000..90eca5a8b2799
    --- /dev/null
    +++ b/gradle/findbugs-exclude.xml
    @@ -0,0 +1,28 @@
    +
    +
    +
    +    
    +    
    +        
    +    
    +
    
    From 0fba529608a5eb829feb66a499c89ead40b79694 Mon Sep 17 00:00:00 2001
    From: Damian Guy 
    Date: Wed, 1 Mar 2017 11:21:41 -0800
    Subject: [PATCH 049/101] KAFKA-4677 Follow Up: add optimization to
     StickyTaskAssignor for rolling rebounce
    
    Detect when a rebalance has happened due to one or more existing nodes bouncing. Keep assignment of previous active tasks the same and only assign the tasks that were not active to the new clients.
    
    Author: Damian Guy 
    
    Reviewers: Guozhang Wang 
    
    Closes #2609 from dguy/kstreams-575
    ---
     .../internals/assignment/ClientState.java     |   4 +
     .../assignment/StickyTaskAssignor.java        |  71 +++++++---
     .../internals/assignment/ClientStateTest.java |  13 ++
     .../assignment/StickyTaskAssignorTest.java    | 121 +++++++++++++++++-
     4 files changed, 182 insertions(+), 27 deletions(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
    index 3e9a521d94cfc..d5f8ccf1280e6 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
    @@ -150,4 +150,8 @@ Set previousAssignedTasks() {
         int capacity() {
             return capacity;
         }
    +
    +    boolean hasUnfulfilledQuota(final int tasksPerThread) {
    +        return activeTasks.size() < capacity * tasksPerThread;
    +    }
     }
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
    index f06ecaee6973b..81c9305d369fb 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java
    @@ -23,6 +23,7 @@
     import java.util.Collection;
     import java.util.HashMap;
     import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.Map;
     import java.util.Objects;
     import java.util.Set;
    @@ -35,16 +36,12 @@ public class StickyTaskAssignor implements TaskAssignor {
         private final Map previousActiveTaskAssignment = new HashMap<>();
         private final Map> previousStandbyTaskAssignment = new HashMap<>();
         private final TaskPairs taskPairs;
    -    private final int availableCapacity;
    -    private final boolean hasNewTasks;
     
         public StickyTaskAssignor(final Map> clients, final Set taskIds) {
             this.clients = clients;
             this.taskIds = taskIds;
    -        this.availableCapacity = sumCapacity(clients.values());
             taskPairs = new TaskPairs(taskIds.size() * (taskIds.size() - 1) / 2);
             mapPreviousTaskAssignment(clients);
    -        this.hasNewTasks = !previousActiveTaskAssignment.keySet().containsAll(taskIds);
         }
     
         @Override
    @@ -66,35 +63,69 @@ private void assignStandby(final int numStandbyReplicas) {
                                  numStandbyReplicas, taskId);
                         break;
                     }
    -                assign(taskId, ids, false);
    +                allocateTaskWithClientCandidates(taskId, ids, false);
                 }
             }
         }
     
         private void assignActive() {
    -        final Set previouslyAssignedTaskIds = new HashSet<>(previousActiveTaskAssignment.keySet());
    -        previouslyAssignedTaskIds.addAll(previousStandbyTaskAssignment.keySet());
    -        previouslyAssignedTaskIds.retainAll(taskIds);
    -
    -        // assign previously assigned tasks first
    -        for (final TaskId taskId : previouslyAssignedTaskIds) {
    -            assign(taskId, clients.keySet(), true);
    +        final int totalCapacity = sumCapacity(clients.values());
    +        final int tasksPerThread = taskIds.size() / totalCapacity;
    +        final Set assigned = new HashSet<>();
    +
    +        // first try and re-assign existing active tasks to clients that previously had
    +        // the same active task
    +        for (final Map.Entry entry : previousActiveTaskAssignment.entrySet()) {
    +            final TaskId taskId = entry.getKey();
    +            if (taskIds.contains(taskId)) {
    +                final ClientState client = clients.get(entry.getValue());
    +                if (client.hasUnfulfilledQuota(tasksPerThread)) {
    +                    assignTaskToClient(assigned, taskId, client);
    +                }
    +            }
             }
     
    -        final Set newTasks  = new HashSet<>(taskIds);
    -        newTasks.removeAll(previouslyAssignedTaskIds);
    +        final Set unassigned = new HashSet<>(taskIds);
    +        unassigned.removeAll(assigned);
    +
    +        // try and assign any remaining unassigned tasks to clients that previously
    +        // have seen the task.
    +        for (final Iterator iterator = unassigned.iterator(); iterator.hasNext(); ) {
    +            final TaskId taskId = iterator.next();
    +            final Set clientIds = previousStandbyTaskAssignment.get(taskId);
    +            if (clientIds != null) {
    +                for (final ID clientId : clientIds) {
    +                    final ClientState client = clients.get(clientId);
    +                    if (client.hasUnfulfilledQuota(tasksPerThread)) {
    +                        assignTaskToClient(assigned, taskId, client);
    +                        iterator.remove();
    +                        break;
    +                    }
    +                }
    +            }
    +        }
     
    -        for (final TaskId taskId : newTasks) {
    -            assign(taskId, clients.keySet(), true);
    +        // assign any remaining unassigned tasks
    +        for (final TaskId taskId : unassigned) {
    +            allocateTaskWithClientCandidates(taskId, clients.keySet(), true);
             }
    +
         }
     
    -    private void assign(final TaskId taskId, final Set clientsWithin, final boolean active) {
    +
    +
    +    private void allocateTaskWithClientCandidates(final TaskId taskId, final Set clientsWithin, final boolean active) {
             final ClientState client = findClient(taskId, clientsWithin);
             taskPairs.addPairs(taskId, client.assignedTasks());
             client.assign(taskId, active);
         }
     
    +    private void assignTaskToClient(final Set assigned, final TaskId taskId, final ClientState client) {
    +        taskPairs.addPairs(taskId, client.assignedTasks());
    +        client.assign(taskId, true);
    +        assigned.add(taskId);
    +    }
    +
         private Set findClientsWithoutAssignedTask(final TaskId taskId) {
             final Set clientIds = new HashSet<>();
             for (final Map.Entry> client : clients.entrySet()) {
    @@ -131,9 +162,7 @@ private ClientState findClient(final TaskId taskId,
         }
     
         private boolean shouldBalanceLoad(final ClientState client) {
    -        return !hasNewTasks
    -                && client.reachedCapacity()
    -                && hasClientsWithMoreAvailableCapacity(client);
    +        return client.reachedCapacity() && hasClientsWithMoreAvailableCapacity(client);
         }
     
         private boolean hasClientsWithMoreAvailableCapacity(final ClientState client) {
    @@ -208,6 +237,7 @@ private void mapPreviousTaskAssignment(final Map> client
                     previousStandbyTaskAssignment.get(prevAssignedTask).add(clientState.getKey());
                 }
             }
    +
         }
     
         private int sumCapacity(final Collection> values) {
    @@ -218,7 +248,6 @@ private int sumCapacity(final Collection> values) {
             return capacity;
         }
     
    -
         private static class TaskPairs {
             private final Set pairs;
             private final int maxPairs;
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java
    index 669284446213b..af2c9e34d64be 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java
    @@ -146,5 +146,18 @@ public void shouldThrowIllegalStateExceptionIfCapacityOfOtherClientStateIsZero()
             c1.hasMoreAvailableCapacityThan(new ClientState(0));
         }
     
    +    @Test
    +    public void shouldHaveUnfulfilledQuotaWhenActiveTaskSizeLessThanCapacityTimesTasksPerThread() throws Exception {
    +        final ClientState client = new ClientState<>(1);
    +        client.assign(1, true);
    +        assertTrue(client.hasUnfulfilledQuota(2));
    +    }
    +
    +    @Test
    +    public void shouldNotHaveUnfulfilledQuotaWhenActiveTaskSizeGreaterEqualThanCapacityTimesTasksPerThread() throws Exception {
    +        final ClientState client = new ClientState<>(1);
    +        client.assign(1, true);
    +        assertFalse(client.hasUnfulfilledQuota(1));
    +    }
     
     }
    \ No newline at end of file
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java
    index a782ea3888228..f37bf7d6a9a0c 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java
    @@ -31,6 +31,7 @@
     
     import static org.hamcrest.CoreMatchers.equalTo;
     import static org.hamcrest.MatcherAssert.assertThat;
    +import static org.hamcrest.core.IsCollectionContaining.hasItem;
     import static org.hamcrest.core.IsCollectionContaining.hasItems;
     import static org.hamcrest.core.IsNot.not;
     import static org.junit.Assert.assertTrue;
    @@ -41,6 +42,8 @@ public class StickyTaskAssignorTest {
         private final TaskId task01 = new TaskId(0, 1);
         private final TaskId task02 = new TaskId(0, 2);
         private final TaskId task03 = new TaskId(0, 3);
    +    private final TaskId task04 = new TaskId(0, 4);
    +    private final TaskId task05 = new TaskId(0, 5);
         private final Map> clients = new TreeMap<>();
         private final Integer p1 = 1;
         private final Integer p2 = 2;
    @@ -448,9 +451,6 @@ public void shouldMoveMinimalNumberOfTasksWhenPreviouslyAboveCapacityAndNewClien
     
         @Test
         public void shouldNotMoveAnyTasksWhenNewTasksAdded() throws Exception {
    -        final TaskId task04 = new TaskId(0, 4);
    -        final TaskId task05 = new TaskId(0, 5);
    -
             createClientWithPreviousActiveTasks(p1, 1, task00, task01);
             createClientWithPreviousActiveTasks(p2, 1, task02, task03);
     
    @@ -463,8 +463,6 @@ public void shouldNotMoveAnyTasksWhenNewTasksAdded() throws Exception {
     
         @Test
         public void shouldAssignNewTasksToNewClientWhenPreviousTasksAssignedToOldClients() throws Exception {
    -        final TaskId task04 = new TaskId(0, 4);
    -        final TaskId task05 = new TaskId(0, 5);
     
             createClientWithPreviousActiveTasks(p1, 1, task02, task01);
             createClientWithPreviousActiveTasks(p2, 1, task00, task03);
    @@ -478,9 +476,120 @@ public void shouldAssignNewTasksToNewClientWhenPreviousTasksAssignedToOldClients
             assertThat(clients.get(p3).activeTasks(), hasItems(task04, task05));
         }
     
    +    @Test
    +    public void shouldAssignTasksNotPreviouslyActiveToNewClient() throws Exception {
    +        final TaskId task10 = new TaskId(0, 10);
    +        final TaskId task11 = new TaskId(0, 11);
    +        final TaskId task12 = new TaskId(1, 2);
    +        final TaskId task13 = new TaskId(1, 3);
    +        final TaskId task20 = new TaskId(2, 0);
    +        final TaskId task21 = new TaskId(2, 1);
    +        final TaskId task22 = new TaskId(2, 2);
    +        final TaskId task23 = new TaskId(2, 3);
    +
    +        final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task12, task13);
    +        c1.addPreviousStandbyTasks(Utils.mkSet(task00, task11, task20, task21, task23));
    +        final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task00, task11, task22);
    +        c2.addPreviousStandbyTasks(Utils.mkSet(task01, task10, task02, task20, task03, task12, task21, task13, task23));
    +        final ClientState c3 = createClientWithPreviousActiveTasks(p3, 1, task20, task21, task23);
    +        c3.addPreviousStandbyTasks(Utils.mkSet(task02, task12));
    +
    +        final ClientState newClient = createClient(p4, 1);
    +        newClient.addPreviousStandbyTasks(Utils.mkSet(task00, task10, task01, task02, task11, task20, task03, task12, task21, task13, task22, task23));
    +
    +        final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task10, task01, task02, task11, task20, task03, task12, task21, task13, task22, task23);
    +        taskAssignor.assign(0);
    +
    +        assertThat(c1.activeTasks(), equalTo(Utils.mkSet(task01, task12, task13)));
    +        assertThat(c2.activeTasks(), equalTo(Utils.mkSet(task00, task11, task22)));
    +        assertThat(c3.activeTasks(), equalTo(Utils.mkSet(task20, task21, task23)));
    +        assertThat(newClient.activeTasks(), equalTo(Utils.mkSet(task02, task03, task10)));
    +    }
    +
    +    @Test
    +    public void shouldAssignTasksNotPreviouslyActiveToMultipleNewClients() throws Exception {
    +        final TaskId task10 = new TaskId(0, 10);
    +        final TaskId task11 = new TaskId(0, 11);
    +        final TaskId task12 = new TaskId(1, 2);
    +        final TaskId task13 = new TaskId(1, 3);
    +        final TaskId task20 = new TaskId(2, 0);
    +        final TaskId task21 = new TaskId(2, 1);
    +        final TaskId task22 = new TaskId(2, 2);
    +        final TaskId task23 = new TaskId(2, 3);
    +
    +        final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task12, task13);
    +        c1.addPreviousStandbyTasks(Utils.mkSet(task00, task11, task20, task21, task23));
    +        final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task00, task11, task22);
    +        c2.addPreviousStandbyTasks(Utils.mkSet(task01, task10, task02, task20, task03, task12, task21, task13, task23));
    +
    +        final ClientState bounce1 = createClient(p3, 1);
    +        bounce1.addPreviousStandbyTasks(Utils.mkSet(task20, task21, task23));
    +
    +        final ClientState bounce2 = createClient(p4, 1);
    +        bounce2.addPreviousStandbyTasks(Utils.mkSet(task02, task03, task10));
    +
    +        final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task10, task01, task02, task11, task20, task03, task12, task21, task13, task22, task23);
    +        taskAssignor.assign(0);
    +
    +        assertThat(c1.activeTasks(), equalTo(Utils.mkSet(task01, task12, task13)));
    +        assertThat(c2.activeTasks(), equalTo(Utils.mkSet(task00, task11, task22)));
    +        assertThat(bounce1.activeTasks(), equalTo(Utils.mkSet(task20, task21, task23)));
    +        assertThat(bounce2.activeTasks(), equalTo(Utils.mkSet(task02, task03, task10)));
    +    }
    +
    +    @Test
    +    public void shouldAssignTasksToNewClient() throws Exception {
    +        createClientWithPreviousActiveTasks(p1, 1, task01, task02);
    +        createClient(p2, 1);
    +        createTaskAssignor(task01, task02).assign(0);
    +        assertThat(clients.get(p1).activeTaskCount(), equalTo(1));
    +    }
    +
    +    @Test
    +    public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExistingClients() throws Exception {
    +        final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00, task01, task02);
    +        final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task03, task04, task05);
    +        final ClientState newClient = createClient(p3, 1);
    +
    +        final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task01, task02, task03, task04, task05);
    +        taskAssignor.assign(0);
    +        assertThat(c1.activeTasks(), not(hasItem(task03)));
    +        assertThat(c1.activeTasks(), not(hasItem(task04)));
    +        assertThat(c1.activeTasks(), not(hasItem(task05)));
    +        assertThat(c1.activeTaskCount(), equalTo(2));
    +        assertThat(c2.activeTasks(), not(hasItems(task00)));
    +        assertThat(c2.activeTasks(), not(hasItems(task01)));
    +        assertThat(c2.activeTasks(), not(hasItems(task02)));
    +        assertThat(c2.activeTaskCount(), equalTo(2));
    +        assertThat(newClient.activeTaskCount(), equalTo(2));
    +    }
    +
    +    @Test
    +    public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExistingAndBouncedClients() throws Exception {
    +        final TaskId task06 = new TaskId(0, 6);
    +        final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00, task01, task02, task06);
    +        final ClientState c2 = createClient(p2, 1);
    +        c2.addPreviousStandbyTasks(Utils.mkSet(task03, task04, task05));
    +        final ClientState newClient = createClient(p3, 1);
    +
    +        final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task01, task02, task03, task04, task05, task06);
    +        taskAssignor.assign(0);
    +        assertThat(c1.activeTasks(), not(hasItem(task03)));
    +        assertThat(c1.activeTasks(), not(hasItem(task04)));
    +        assertThat(c1.activeTasks(), not(hasItem(task05)));
    +        assertThat(c1.activeTaskCount(), equalTo(3));
    +        assertThat(c2.activeTasks(), not(hasItems(task00)));
    +        assertThat(c2.activeTasks(), not(hasItems(task01)));
    +        assertThat(c2.activeTasks(), not(hasItems(task02)));
    +        assertThat(c2.activeTaskCount(), equalTo(2));
    +        assertThat(newClient.activeTaskCount(), equalTo(2));
    +    }
    +
         private StickyTaskAssignor createTaskAssignor(final TaskId... tasks) {
    +        final List taskIds = Arrays.asList(tasks);
    +        Collections.shuffle(taskIds);
             return new StickyTaskAssignor<>(clients,
    -                                        new HashSet<>(Arrays.asList(tasks)));
    +                                        new HashSet<>(taskIds));
         }
     
         private List allActiveTasks() {
    
    From b380a82d5be7c68141590467911ecb61db45ed1e Mon Sep 17 00:00:00 2001
    From: Damian Guy 
    Date: Wed, 1 Mar 2017 12:07:46 -0800
    Subject: [PATCH 050/101] MINOR: improve MinTimestampTrackerTest and fix NPE
     when null element removed
    
    Author: Damian Guy 
    
    Reviewers: Matthias J. Sax, Guozhang Wang
    
    Closes #2611 from dguy/testing
    ---
     .../internals/MinTimestampTracker.java        |  15 ++-
     .../internals/MinTimestampTrackerTest.java    | 102 ++++++++----------
     2 files changed, 55 insertions(+), 62 deletions(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java
    index ef7d99040410c..a67675c7ad7de 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java
    @@ -34,7 +34,7 @@ public class MinTimestampTracker implements TimestampTracker {
         /**
          * @throws NullPointerException if the element is null
          */
    -    public void addElement(Stamped elem) {
    +    public void addElement(final Stamped elem) {
             if (elem == null) throw new NullPointerException();
     
             Stamped minElem = descendingSubsequence.peekLast();
    @@ -45,12 +45,19 @@ public void addElement(Stamped elem) {
             descendingSubsequence.offerLast(elem);
         }
     
    -    public void removeElement(Stamped elem) {
    -        if (elem != null && descendingSubsequence.peekFirst() == elem)
    +    public void removeElement(final Stamped elem) {
    +        if (elem == null) {
    +            return;
    +        }
    +
    +        if (descendingSubsequence.peekFirst() == elem) {
                 descendingSubsequence.removeFirst();
    +        }
     
    -        if (descendingSubsequence.isEmpty())
    +        if (descendingSubsequence.isEmpty()) {
                 lastKnownTime = elem.timestamp;
    +        }
    +
         }
     
         public int size() {
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java
    index c398dc523d016..f6a1518066fec 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java
    @@ -16,77 +16,63 @@
      */
     package org.apache.kafka.streams.processor.internals;
     
    -import static org.junit.Assert.assertEquals;
    +import static org.hamcrest.CoreMatchers.equalTo;
    +import static org.hamcrest.MatcherAssert.assertThat;
     
     import org.junit.Test;
     
     public class MinTimestampTrackerTest {
     
    -    private Stamped elem(long timestamp) {
    -        return new Stamped<>("", timestamp);
    -    }
    +    private MinTimestampTracker tracker = new MinTimestampTracker<>();
     
    -    @SuppressWarnings("unchecked")
         @Test
    -    public void testTracking() {
    -        TimestampTracker tracker = new MinTimestampTracker<>();
    -
    -        Object[] elems = new Object[]{
    -            elem(100), elem(101), elem(102), elem(98), elem(99), elem(100)
    -        };
    -
    -        int insertionIndex = 0;
    -        int removalIndex = 0;
    -
    -        // add 100
    -        tracker.addElement((Stamped) elems[insertionIndex++]);
    -        assertEquals(100L, tracker.get());
    -
    -        // add 101
    -        tracker.addElement((Stamped) elems[insertionIndex++]);
    -        assertEquals(100L, tracker.get());
    -
    -        // remove 100
    -        tracker.removeElement((Stamped) elems[removalIndex++]);
    -        assertEquals(101L, tracker.get());
    -
    -        // add 102
    -        tracker.addElement((Stamped) elems[insertionIndex++]);
    -        assertEquals(101L, tracker.get());
    -
    -        // add 98
    -        tracker.addElement((Stamped) elems[insertionIndex++]);
    -        assertEquals(98L, tracker.get());
    -
    -        // add 99
    -        tracker.addElement((Stamped) elems[insertionIndex++]);
    -        assertEquals(98L, tracker.get());
    -
    -        // add 100
    -        tracker.addElement((Stamped) elems[insertionIndex++]);
    -        assertEquals(98L, tracker.get());
    +    public void shouldReturnNotKnownTimestampWhenNoRecordsEverAdded() throws Exception {
    +        assertThat(tracker.get(), equalTo(TimestampTracker.NOT_KNOWN));
    +    }
     
    -        // remove 101
    -        tracker.removeElement((Stamped) elems[removalIndex++]);
    -        assertEquals(98L, tracker.get());
    +    @Test
    +    public void shouldReturnTimestampOfOnlyRecord() throws Exception {
    +        tracker.addElement(elem(100));
    +        assertThat(tracker.get(), equalTo(100L));
    +    }
     
    -        // remove 102
    -        tracker.removeElement((Stamped) elems[removalIndex++]);
    -        assertEquals(98L, tracker.get());
    +    @Test
    +    public void shouldReturnLowestAvailableTimestampFromAllInputs() throws Exception {
    +        tracker.addElement(elem(100));
    +        tracker.addElement(elem(99));
    +        tracker.addElement(elem(102));
    +        assertThat(tracker.get(), equalTo(99L));
    +    }
     
    -        // remove 98
    -        tracker.removeElement((Stamped) elems[removalIndex++]);
    -        assertEquals(99L, tracker.get());
    +    @Test
    +    public void shouldReturnLowestAvailableTimestampAfterPreviousLowestRemoved() throws Exception {
    +        final Stamped lowest = elem(88);
    +        tracker.addElement(lowest);
    +        tracker.addElement(elem(101));
    +        tracker.addElement(elem(99));
    +        tracker.removeElement(lowest);
    +        assertThat(tracker.get(), equalTo(99L));
    +    }
     
    -        // remove 99
    -        tracker.removeElement((Stamped) elems[removalIndex++]);
    -        assertEquals(100L, tracker.get());
    +    @Test
    +    public void shouldReturnLastKnownTimestampWhenAllElementsHaveBeenRemoved() throws Exception {
    +        final Stamped record = elem(98);
    +        tracker.addElement(record);
    +        tracker.removeElement(record);
    +        assertThat(tracker.get(), equalTo(98L));
    +    }
     
    -        // remove 100
    -        tracker.removeElement((Stamped) elems[removalIndex++]);
    -        assertEquals(100L, tracker.get());
    +    @Test
    +    public void shouldIgnoreNullRecordOnRemove() throws Exception {
    +        tracker.removeElement(null);
    +    }
     
    -        assertEquals(insertionIndex, removalIndex);
    +    @Test(expected = NullPointerException.class)
    +    public void shouldThrowNullPointerExceptionWhenTryingToAddNullElement() throws Exception {
    +        tracker.addElement(null);
         }
     
    +    private Stamped elem(final long timestamp) {
    +        return new Stamped<>("", timestamp);
    +    }
     }
    \ No newline at end of file
    
    From ef92bb4e00da10728cf74c2d81f8f2bbec4c9c02 Mon Sep 17 00:00:00 2001
    From: Eno Thereska 
    Date: Wed, 1 Mar 2017 14:36:08 -0800
    Subject: [PATCH 051/101] MINOR: Minor reduce unnecessary calls to
     time.millisecond (part 2)
    
    Avoid calling time.milliseconds more often than necessary. Cleaning and committing logic can use the timestamp at the start of the loop with minimal consequences. 5-10% improvements noticed with request rates of 450K records/second.
    
    Also tidy up benchmark code a bit more.
    
    Author: Eno Thereska 
    Author: Eno Thereska 
    
    Reviewers: Matthias J. Sax, Damian Guy, Guozhang Wang
    
    Closes #2603 from enothereska/minor-reduce-milliseconds2
    ---
     .../processor/internals/StreamThread.java     | 10 +++----
     .../kafka/streams/perf/SimpleBenchmark.java   | 26 +++++++----------
     .../processor/internals/StreamThreadTest.java | 28 +++++++++----------
     3 files changed, 28 insertions(+), 36 deletions(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
    index 7f482009cde52..033dc7321ff1e 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
    @@ -610,10 +610,10 @@ private void runLoop() {
                 } else {
                     requiresPoll = true;
                 }
    -            maybeCommit();
    +            maybeCommit(timerStartedMs);
                 maybeUpdateStandbyTasks();
     
    -            maybeClean();
    +            maybeClean(timerStartedMs);
             }
             log.info("{} Shutting down at user request", logPrefix);
         }
    @@ -682,8 +682,7 @@ private void maybePunctuate(StreamTask task) {
         /**
          * Commit all tasks owned by this thread if specified interval time has elapsed
          */
    -    protected void maybeCommit() {
    -        long now = time.milliseconds();
    +    protected void maybeCommit(final long now) {
     
             if (commitTimeMs >= 0 && lastCommitMs + commitTimeMs < now) {
                 log.info("{} Committing all tasks because the commit interval {}ms has elapsed", logPrefix, commitTimeMs);
    @@ -698,8 +697,7 @@ protected void maybeCommit() {
         /**
          * Cleanup any states of the tasks that have been removed from this thread
          */
    -    protected void maybeClean() {
    -        long now = time.milliseconds();
    +    protected void maybeClean(final long now) {
     
             if (now > lastCleanMs + cleanTimeMs) {
                 stateDirectory.cleanRemovedTasks(cleanTimeMs);
    diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
    index dc1bdf52ebc5c..cf593e2104c6c 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
    @@ -141,7 +141,7 @@ private void run() throws Exception {
                     kTableKTableJoin(JOIN_TOPIC_1_PREFIX + "KTableKTable", JOIN_TOPIC_2_PREFIX + "KTableKTable");
                     break;
                 case "produce":
    -                produce(SOURCE_TOPIC, VALUE_SIZE, "simple-benchmark-produce", numRecords, true, numRecords, true);
    +                produce(SOURCE_TOPIC);
                     break;
                 case "consume":
                     consume(SOURCE_TOPIC);
    @@ -444,7 +444,6 @@ public void produce(String topic) throws Exception {
                 return;
             }
             produce(topic, VALUE_SIZE, "simple-benchmark-produce", numRecords, true, numRecords, true);
    -
         }
         /**
          * Produce values to a topic
    @@ -458,9 +457,10 @@ public void produce(String topic) throws Exception {
          *                   when this produce step is part of another benchmark that produces its own stats
          */
         private void produce(String topic, int valueSizeBytes, String clientId, int numRecords, boolean sequential,
    -                        int upperRange, boolean printStats) throws Exception {
    -
    +                         int upperRange, boolean printStats) throws Exception {
     
    +        processedRecords = 0;
    +        processedBytes = 0;
             if (sequential) {
                 if (upperRange < numRecords) throw new Exception("UpperRange must be >= numRecords");
             }
    @@ -486,17 +486,15 @@ private void produce(String topic, int valueSizeBytes, String clientId, int numR
                 producer.send(new ProducerRecord<>(topic, key, value));
                 if (sequential) key++;
                 else key = rand.nextInt(upperRange);
    +            processedRecords++;
    +            processedBytes += value.length + Integer.SIZE;
             }
             producer.close();
     
             long endTime = System.currentTimeMillis();
     
             if (printStats) {
    -            System.out.println("Producer Performance [records/latency/rec-sec/MB-sec write]: " +
    -                numRecords + "/" +
    -                (endTime - startTime) + "/" +
    -                recordsPerSec(endTime - startTime, numRecords) + "/" +
    -                megabytesPerSec(endTime - startTime, numRecords * valueSizeBytes));
    +            printResults("Producer Performance [records/latency/rec-sec/MB-sec write]: ", endTime - startTime);
             }
         }
     
    @@ -540,11 +538,7 @@ public void consume(String topic) throws Exception {
             long endTime = System.currentTimeMillis();
     
             consumer.close();
    -        System.out.println("Consumer Performance [records/latency/rec-sec/MB-sec read]: " +
    -            processedRecords + "/" +
    -            (endTime - startTime) + "/" +
    -            recordsPerSec(endTime - startTime, processedRecords) + "/" +
    -            megabytesPerSec(endTime - startTime, processedBytes));
    +        printResults("Consumer Performance [records/latency/rec-sec/MB-sec read]: ", endTime - startTime);
         }
     
         private KafkaStreams createKafkaStreams(String topic, final CountDownLatch latch) {
    @@ -734,11 +728,11 @@ public void close() {
         }
     
         private double megabytesPerSec(long time, long processedBytes) {
    -        return  ((double) processedBytes / 1024 / 1024) / (time / 1000.0);
    +        return  (processedBytes / 1024.0 / 1024.0) / (time / 1000.0);
         }
     
         private double recordsPerSec(long time, int numRecords) {
    -        return (double) numRecords / ((double) time / 1000);
    +        return numRecords / (time / 1000.0);
         }
     
         private List getAllPartitions(KafkaConsumer consumer, String... topics) {
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
    index 87b30b2ea6225..e36a236962257 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
    @@ -485,7 +485,7 @@ public void testMaybeClean() throws Exception {
                 stateDir3.mkdir();
                 extraDir.mkdir();
     
    -            MockTime mockTime = new MockTime();
    +            final MockTime mockTime = new MockTime();
     
                 TopologyBuilder builder = new TopologyBuilder().setApplicationId("X");
                 builder.addSource("source1", "topic1");
    @@ -495,8 +495,8 @@ public void testMaybeClean() throws Exception {
                                                        0) {
     
                     @Override
    -                public void maybeClean() {
    -                    super.maybeClean();
    +                public void maybeClean(long now) {
    +                    super.maybeClean(now);
                     }
     
                     @Override
    @@ -547,7 +547,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part
     
                 // all directories should still exit before the cleanup delay time
                 mockTime.sleep(cleanupDelay - 10L);
    -            thread.maybeClean();
    +            thread.maybeClean(mockTime.milliseconds());
                 assertTrue(stateDir1.exists());
                 assertTrue(stateDir2.exists());
                 assertTrue(stateDir3.exists());
    @@ -555,7 +555,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part
     
                 // all state directories except for task task2 & task3 will be removed. the extra directory should still exists
                 mockTime.sleep(11L);
    -            thread.maybeClean();
    +            thread.maybeClean(mockTime.milliseconds());
                 assertTrue(stateDir1.exists());
                 assertTrue(stateDir2.exists());
                 assertFalse(stateDir3.exists());
    @@ -585,7 +585,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part
     
                 // all state directories for task task1 & task2 still exist before the cleanup delay time
                 mockTime.sleep(cleanupDelay - 10L);
    -            thread.maybeClean();
    +            thread.maybeClean(mockTime.milliseconds());
                 assertTrue(stateDir1.exists());
                 assertTrue(stateDir2.exists());
                 assertFalse(stateDir3.exists());
    @@ -593,7 +593,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part
     
                 // all state directories for task task1 & task2 are removed
                 mockTime.sleep(11L);
    -            thread.maybeClean();
    +            thread.maybeClean(mockTime.milliseconds());
                 assertFalse(stateDir1.exists());
                 assertFalse(stateDir2.exists());
                 assertFalse(stateDir3.exists());
    @@ -615,7 +615,7 @@ public void testMaybeCommit() throws Exception {
     
                 StreamsConfig config = new StreamsConfig(props);
     
    -            MockTime mockTime = new MockTime();
    +            final MockTime mockTime = new MockTime();
     
                 TopologyBuilder builder = new TopologyBuilder().setApplicationId("X");
                 builder.addSource("source1", "topic1");
    @@ -625,8 +625,8 @@ public void testMaybeCommit() throws Exception {
                                                        0) {
     
                     @Override
    -                public void maybeCommit() {
    -                    super.maybeCommit();
    +                public void maybeCommit(long now) {
    +                    super.maybeCommit(now);
                     }
     
                     @Override
    @@ -657,14 +657,14 @@ protected StreamTask createStreamTask(TaskId id, Collection part
     
                 // no task is committed before the commit interval
                 mockTime.sleep(commitInterval - 10L);
    -            thread.maybeCommit();
    +            thread.maybeCommit(mockTime.milliseconds());
                 for (StreamTask task : thread.tasks().values()) {
                     assertFalse(((TestStreamTask) task).committed);
                 }
     
                 // all tasks are committed after the commit interval
                 mockTime.sleep(11L);
    -            thread.maybeCommit();
    +            thread.maybeCommit(mockTime.milliseconds());
                 for (StreamTask task : thread.tasks().values()) {
                     assertTrue(((TestStreamTask) task).committed);
                     ((TestStreamTask) task).committed = false;
    @@ -672,14 +672,14 @@ protected StreamTask createStreamTask(TaskId id, Collection part
     
                 // no task is committed before the commit interval, again
                 mockTime.sleep(commitInterval - 10L);
    -            thread.maybeCommit();
    +            thread.maybeCommit(mockTime.milliseconds());
                 for (StreamTask task : thread.tasks().values()) {
                     assertFalse(((TestStreamTask) task).committed);
                 }
     
                 // all tasks are committed after the commit interval, again
                 mockTime.sleep(11L);
    -            thread.maybeCommit();
    +            thread.maybeCommit(mockTime.milliseconds());
                 for (StreamTask task : thread.tasks().values()) {
                     assertTrue(((TestStreamTask) task).committed);
                     ((TestStreamTask) task).committed = false;
    
    From 1b902b4ed39e78066fab163d1b6d54dd435b1d7b Mon Sep 17 00:00:00 2001
    From: huxi 
    Date: Thu, 2 Mar 2017 09:55:01 -0800
    Subject: [PATCH 052/101] KAFKA4811; ReplicaFetchThread may fail to create due
     to existing metric
    
    Have fetcherThreadMap keyed off brokerId + fetcherId instead of broker + fetcherId, but did not consider the case where port is changed.
    
    Author: huxi 
    
    Reviewers: Jun Rao 
    
    Closes #2606 from amethystic/kafka4811_ReplicaFetchThread_fail_create
    ---
     .../kafka/server/AbstractFetcherManager.scala | 29 +++++++++++++------
     .../kafka/server/AbstractFetcherThread.scala  |  2 +-
     2 files changed, 21 insertions(+), 10 deletions(-)
    
    diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
    index 0a17f8eb9772e..2b2aa7b900ee1 100755
    --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
    +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala
    @@ -30,7 +30,7 @@ import org.apache.kafka.common.utils.Utils
     abstract class AbstractFetcherManager(protected val name: String, clientId: String, numFetchers: Int = 1)
       extends Logging with KafkaMetricsGroup {
       // map of (source broker_id, fetcher_id per source broker) => fetcher
    -  private val fetcherThreadMap = new mutable.HashMap[BrokerAndFetcherId, AbstractFetcherThread]
    +  private val fetcherThreadMap = new mutable.HashMap[BrokerIdAndFetcherId, AbstractFetcherThread]
       private val mapLock = new Object
       this.logIdent = "[" + name + "] "
     
    @@ -75,17 +75,26 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
         mapLock synchronized {
           val partitionsPerFetcher = partitionAndOffsets.groupBy { case(topicPartition, brokerAndInitialOffset) =>
             BrokerAndFetcherId(brokerAndInitialOffset.broker, getFetcherId(topicPartition.topic, topicPartition.partition))}
    +
    +      def addAndStartFetcherThread(brokerAndFetcherId: BrokerAndFetcherId, brokerIdAndFetcherId: BrokerIdAndFetcherId) {
    +        val fetcherThread = createFetcherThread(brokerAndFetcherId.fetcherId, brokerAndFetcherId.broker)
    +        fetcherThreadMap.put(brokerIdAndFetcherId, fetcherThread)
    +        fetcherThread.start
    +      }
    +
           for ((brokerAndFetcherId, partitionAndOffsets) <- partitionsPerFetcher) {
    -        var fetcherThread: AbstractFetcherThread = null
    -        fetcherThreadMap.get(brokerAndFetcherId) match {
    -          case Some(f) => fetcherThread = f
    +        val brokerIdAndFetcherId = BrokerIdAndFetcherId(brokerAndFetcherId.broker.id, brokerAndFetcherId.fetcherId)
    +        fetcherThreadMap.get(brokerIdAndFetcherId) match {
    +          case Some(f) if f.sourceBroker.host == brokerAndFetcherId.broker.host && f.sourceBroker.port == brokerAndFetcherId.broker.port =>
    +            // reuse the fetcher thread
    +          case Some(f) =>
    +            f.shutdown()
    +            addAndStartFetcherThread(brokerAndFetcherId, brokerIdAndFetcherId)
               case None =>
    -            fetcherThread = createFetcherThread(brokerAndFetcherId.fetcherId, brokerAndFetcherId.broker)
    -            fetcherThreadMap.put(brokerAndFetcherId, fetcherThread)
    -            fetcherThread.start
    +            addAndStartFetcherThread(brokerAndFetcherId, brokerIdAndFetcherId)
             }
     
    -        fetcherThreadMap(brokerAndFetcherId).addPartitions(partitionAndOffsets.map { case (tp, brokerAndInitOffset) =>
    +        fetcherThreadMap(brokerIdAndFetcherId).addPartitions(partitionAndOffsets.map { case (tp, brokerAndInitOffset) =>
               tp -> brokerAndInitOffset.initOffset
             })
           }
    @@ -105,7 +114,7 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
     
       def shutdownIdleFetcherThreads() {
         mapLock synchronized {
    -      val keysToBeRemoved = new mutable.HashSet[BrokerAndFetcherId]
    +      val keysToBeRemoved = new mutable.HashSet[BrokerIdAndFetcherId]
           for ((key, fetcher) <- fetcherThreadMap) {
             if (fetcher.partitionCount <= 0) {
               fetcher.shutdown()
    @@ -133,3 +142,5 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri
     case class BrokerAndFetcherId(broker: BrokerEndPoint, fetcherId: Int)
     
     case class BrokerAndInitialOffset(broker: BrokerEndPoint, initOffset: Long)
    +
    +case class BrokerIdAndFetcherId(brokerId: Int, fetcherId: Int)
    diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
    index 64629689404bb..0eb3ad83876e6 100755
    --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
    +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
    @@ -44,7 +44,7 @@ import org.apache.kafka.common.record.MemoryRecords
      */
     abstract class AbstractFetcherThread(name: String,
                                          clientId: String,
    -                                     sourceBroker: BrokerEndPoint,
    +                                     val sourceBroker: BrokerEndPoint,
                                          fetchBackOffMs: Int = 0,
                                          isInterruptible: Boolean = true)
       extends ShutdownableThread(name, isInterruptible) {
    
    From ca06862a7005ca476f900bd9c2373021422d695b Mon Sep 17 00:00:00 2001
    From: Stevo Slavic 
    Date: Thu, 2 Mar 2017 13:12:59 -0800
    Subject: [PATCH 053/101] KAFKA-2358: Cluster collection returning methods
     never return null
    
    See https://issues.apache.org/jira/browse/KAFKA-2358
    
    Author: Stevo Slavic 
    
    Reviewers: Jason Gustafson, Guozhang Wang
    
    Closes #96 from sslavic/feature/KAFKA-2358
    ---
     .../src/main/java/org/apache/kafka/clients/Metadata.java | 2 +-
     .../org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +-
     .../org/apache/kafka/clients/producer/MockProducer.java  | 2 +-
     .../src/main/java/org/apache/kafka/common/Cluster.java   | 9 ++++++---
     .../test/java/org/apache/kafka/clients/MetadataTest.java | 2 +-
     .../kafka/streams/processor/DefaultPartitionGrouper.java | 4 ++--
     .../processor/internals/StreamPartitionAssignor.java     | 2 +-
     .../processor/internals/StreamsMetadataState.java        | 2 +-
     8 files changed, 14 insertions(+), 11 deletions(-)
    
    diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
    index 65da33054962a..87e5862727758 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
    @@ -329,7 +329,7 @@ private Cluster getClusterForCurrentTopics(Cluster cluster) {
     
                 for (String topic : this.topics.keySet()) {
                     List partitionInfoList = cluster.partitionsForTopic(topic);
    -                if (partitionInfoList != null) {
    +                if (!partitionInfoList.isEmpty()) {
                         partitionInfos.addAll(partitionInfoList);
                     }
                 }
    diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
    index 63a39faf4481d..1e3749757595c 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
    @@ -1342,7 +1342,7 @@ public List partitionsFor(String topic) {
             try {
                 Cluster cluster = this.metadata.fetch();
                 List parts = cluster.partitionsForTopic(topic);
    -            if (parts != null)
    +            if (!parts.isEmpty())
                     return parts;
     
                 Map> topicMetadata = fetcher.getTopicMetadata(
    diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
    index bafb04839fd00..35f5d940e0364 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
    @@ -114,7 +114,7 @@ public synchronized Future send(ProducerRecord record) {
         @Override
         public synchronized Future send(ProducerRecord record, Callback callback) {
             int partition = 0;
    -        if (this.cluster.partitionsForTopic(record.topic()) != null)
    +        if (!this.cluster.partitionsForTopic(record.topic()).isEmpty())
                 partition = partition(record, this.cluster);
             TopicPartition topicPartition = new TopicPartition(record.topic(), partition);
             ProduceRequestResult result = new ProduceRequestResult(topicPartition);
    diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java
    index b7408e3ad6bfd..ba1d2af48135e 100644
    --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java
    +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java
    @@ -207,7 +207,8 @@ public PartitionInfo partition(TopicPartition topicPartition) {
          * @return A list of partitions
          */
         public List partitionsForTopic(String topic) {
    -        return this.partitionsByTopic.get(topic);
    +        List parts = this.partitionsByTopic.get(topic);
    +        return (parts == null) ? Collections.emptyList() : parts;
         }
     
         /**
    @@ -226,7 +227,8 @@ public Integer partitionCountForTopic(String topic) {
          * @return A list of partitions
          */
         public List availablePartitionsForTopic(String topic) {
    -        return this.availablePartitionsByTopic.get(topic);
    +        List parts = this.availablePartitionsByTopic.get(topic);
    +        return (parts == null) ? Collections.emptyList() : parts;
         }
     
         /**
    @@ -235,7 +237,8 @@ public List availablePartitionsForTopic(String topic) {
          * @return A list of partitions
          */
         public List partitionsForNode(int nodeId) {
    -        return this.partitionsByNode.get(nodeId);
    +        List parts = this.partitionsByNode.get(nodeId);
    +        return (parts == null) ? Collections.emptyList() : parts;
         }
     
         /**
    diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
    index 1a05abc1e5b50..084ccd8650d87 100644
    --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
    +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
    @@ -412,7 +412,7 @@ public void testNonExpiringMetadata() throws Exception {
         private Thread asyncFetch(final String topic, final long maxWaitMs) {
             Thread thread = new Thread() {
                 public void run() {
    -                while (metadata.fetch().partitionsForTopic(topic) == null) {
    +                while (metadata.fetch().partitionsForTopic(topic).isEmpty()) {
                         try {
                             metadata.awaitUpdate(metadata.requestUpdate(), maxWaitMs);
                         } catch (Exception e) {
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java
    index 5e4da4b049348..19e480908d2be 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java
    @@ -62,7 +62,7 @@ public Map> partitionGroups(Map
     
                     for (String topic : topicGroup) {
                         List partitions = metadata.partitionsForTopic(topic);
    -                    if (partitions != null && partitionId < partitions.size()) {
    +                    if (partitionId < partitions.size()) {
                             group.add(new TopicPartition(topic, partitionId));
                         }
                     }
    @@ -81,7 +81,7 @@ protected int maxNumPartitions(Cluster metadata, Set topics) {
             for (String topic : topics) {
                 List partitions = metadata.partitionsForTopic(topic);
     
    -            if (partitions == null) {
    +            if (partitions.isEmpty()) {
                     log.info("Skipping assigning topic {} to tasks since its metadata is not available yet", topic);
                     return StreamPartitionAssignor.NOT_AVAILABLE;
                 } else {
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
    index 21b9109285318..e3f66984adb7f 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
    @@ -406,7 +406,7 @@ public Map assign(Cluster metadata, Map partitionInfoList = metadataWithInternalTopics.partitionsForTopic(topic);
    -            if (partitionInfoList != null) {
    +            if (!partitionInfoList.isEmpty()) {
                     for (PartitionInfo partitionInfo : partitionInfoList) {
                         TopicPartition partition = new TopicPartition(partitionInfo.topic(), partitionInfo.partition());
                         if (!allAssignedPartitions.contains(partition)) {
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java
    index 67a26bfa30367..bb74b4803589e 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java
    @@ -279,7 +279,7 @@ private SourceTopicsInfo(final List sourceTopics) {
                 this.sourceTopics = sourceTopics;
                 for (String topic : sourceTopics) {
                     final List partitions = clusterMetadata.partitionsForTopic(topic);
    -                if (partitions != null && partitions.size() > maxPartitions) {
    +                if (partitions.size() > maxPartitions) {
                         maxPartitions = partitions.size();
                         topicWithMostPartitions = partitions.get(0).topic();
                     }
    
    From 9e65b25e9fca6c26cef3498ff747879d4f527700 Mon Sep 17 00:00:00 2001
    From: Bill Bejeck 
    Date: Thu, 2 Mar 2017 13:19:08 -0800
    Subject: [PATCH 054/101] KAFKA-4652: Added tests for KStreamBuilder
    
    Author: bbejeck 
    
    Reviewers: Damian Guy, Guozhang Wang
    
    Closes #2597 from bbejeck/KAFKA-4652_improve_kstream_builder_test_coverage
    ---
     .../streams/kstream/KStreamBuilderTest.java   | 92 +++++++++++++++++++
     1 file changed, 92 insertions(+)
    
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
    index 3c66dc996a803..7ce0b5414d05e 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
    @@ -16,11 +16,13 @@
      */
     package org.apache.kafka.streams.kstream;
     
    +import org.apache.kafka.common.serialization.Serde;
     import org.apache.kafka.common.serialization.Serdes;
     import org.apache.kafka.common.utils.Utils;
     import org.apache.kafka.streams.kstream.internals.KStreamImpl;
     import org.apache.kafka.streams.errors.TopologyBuilderException;
     import org.apache.kafka.streams.processor.StateStore;
    +import org.apache.kafka.streams.processor.TopologyBuilder;
     import org.apache.kafka.streams.processor.internals.ProcessorTopology;
     import org.apache.kafka.test.KStreamTestDriver;
     import org.apache.kafka.test.MockKeyValueMapper;
    @@ -35,8 +37,10 @@
     import java.util.Collections;
     import java.util.Map;
     import java.util.Set;
    +import java.util.regex.Pattern;
     
     import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertFalse;
     import static org.junit.Assert.assertTrue;
     
     public class KStreamBuilderTest {
    @@ -232,4 +236,92 @@ public void shouldMapStateStoresToCorrectSourceTopics() throws Exception {
             assertEquals(Collections.singletonList("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store"));
             assertEquals(Collections.singletonList(APP_ID + "-KSTREAM-MAP-0000000003-repartition"), builder.stateStoreNameToSourceTopics().get("count"));
         }
    +
    +    @Test
    +    public void shouldAddTopicToEarliestAutoOffsetResetList() {
    +        final String topicName = "topic-1";
    +        
    +        builder.stream(TopologyBuilder.AutoOffsetReset.EARLIEST, topicName);
    +        
    +        assertTrue(builder.earliestResetTopicsPattern().matcher(topicName).matches());
    +        assertFalse(builder.latestResetTopicsPattern().matcher(topicName).matches());
    +    }
    +
    +    @Test
    +    public void shouldAddTopicToLatestAutoOffsetResetList() {
    +        final String topicName = "topic-1";
    +
    +        builder.stream(TopologyBuilder.AutoOffsetReset.LATEST, topicName);
    +
    +        assertTrue(builder.latestResetTopicsPattern().matcher(topicName).matches());
    +        assertFalse(builder.earliestResetTopicsPattern().matcher(topicName).matches());
    +    }
    +
    +    @Test
    +    public void shouldAddTableToEarliestAutoOffsetResetList() {
    +        final String topicName = "topic-1";
    +        final String storeName = "test-store";
    +
    +        builder.table(TopologyBuilder.AutoOffsetReset.EARLIEST, topicName, storeName);
    +
    +        assertTrue(builder.earliestResetTopicsPattern().matcher(topicName).matches());
    +        assertFalse(builder.latestResetTopicsPattern().matcher(topicName).matches());
    +    }
    +
    +    @Test
    +    public void shouldAddTableToLatestAutoOffsetResetList() {
    +        final String topicName = "topic-1";
    +        final String storeName = "test-store";
    +
    +        builder.table(TopologyBuilder.AutoOffsetReset.LATEST, topicName, storeName);
    +
    +        assertTrue(builder.latestResetTopicsPattern().matcher(topicName).matches());
    +        assertFalse(builder.earliestResetTopicsPattern().matcher(topicName).matches());
    +    }
    +
    +    @Test
    +    public void shouldNotAddTableToOffsetResetLists() {
    +        final String topicName = "topic-1";
    +        final String storeName = "test-store";
    +        final Serde stringSerde = Serdes.String();
    +
    +        builder.table(stringSerde, stringSerde, topicName, storeName);
    +
    +        assertFalse(builder.latestResetTopicsPattern().matcher(topicName).matches());
    +        assertFalse(builder.earliestResetTopicsPattern().matcher(topicName).matches());
    +    }
    +
    +    @Test
    +    public void shouldNotAddRegexTopicsToOffsetResetLists() {
    +        final Pattern topicPattern = Pattern.compile("topic-\\d");
    +        final String topic = "topic-5";
    +
    +        builder.stream(topicPattern);
    +        
    +        assertFalse(builder.latestResetTopicsPattern().matcher(topic).matches());
    +        assertFalse(builder.earliestResetTopicsPattern().matcher(topic).matches());
    +
    +    }
    +
    +    @Test
    +    public void shouldAddRegexTopicToEarliestAutoOffsetResetList() {
    +        final Pattern topicPattern = Pattern.compile("topic-\\d+");
    +        final String topicTwo = "topic-500000";
    +
    +        builder.stream(TopologyBuilder.AutoOffsetReset.EARLIEST, topicPattern);
    +
    +        assertTrue(builder.earliestResetTopicsPattern().matcher(topicTwo).matches());
    +        assertFalse(builder.latestResetTopicsPattern().matcher(topicTwo).matches());
    +    }
    +
    +    @Test
    +    public void shouldAddRegexTopicToLatestAutoOffsetResetList() {
    +        final Pattern topicPattern = Pattern.compile("topic-\\d+");
    +        final String topicTwo = "topic-1000000";
    +
    +        builder.stream(TopologyBuilder.AutoOffsetReset.LATEST, topicPattern);
    +
    +        assertTrue(builder.latestResetTopicsPattern().matcher(topicTwo).matches());
    +        assertFalse(builder.earliestResetTopicsPattern().matcher(topicTwo).matches());
    +    }
     }
    
    From 59779aa1546c6039ea145c6e21706cbf68cad4a2 Mon Sep 17 00:00:00 2001
    From: "Michael G. Noll" 
    Date: Thu, 2 Mar 2017 13:23:54 -0800
    Subject: [PATCH 055/101] MINOR: Fix typo in GlobalKTable javadocs
    
    Author: Michael G. Noll 
    
    Reviewers: Matthias J. Sax, Damian Guy, Guozhang Wang
    
    Closes #2626 from miguno/trunk-globalktable-typos
    ---
     .../java/org/apache/kafka/streams/kstream/GlobalKTable.java   | 4 ++--
     1 file changed, 2 insertions(+), 2 deletions(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java
    index 3bd4f850e0f46..4ec2ff9d7c4ed 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java
    @@ -52,8 +52,8 @@
      * ReadOnlyKeyValueStore view = streams.store("g1-store", QueryableStoreTypes.keyValueStore());
      * view.get(key); // can be done on any key, as all keys are present
      *}
    - * Note that in contrast to {@link KTable} a {@code GlobalKTable}'s state holds a full copy if the underlying topic and - * thus, all keys can be queried locally. + * Note that in contrast to {@link KTable} a {@code GlobalKTable}'s state holds a full copy of the underlying topic, + * thus all keys can be queried locally. * * @param Type of primary keys * @param Type of value changes From 670980ef29fb70838dcd462d11789c2519981164 Mon Sep 17 00:00:00 2001 From: Clemens Valiente Date: Thu, 2 Mar 2017 13:26:30 -0800 Subject: [PATCH 056/101] KAFKA-4800: Streams State transition ASCII diagrams need fixing and polishing added \
     tags to not break javadoc display of the ASCII diagrams.
    see broken ascii here:
    https://kafka.apache.org/0102/javadoc/org/apache/kafka/streams/KafkaStreams.State.html
    
    fix can be checked with gradle :streams:javadoc and then checking streams/build/docs/javadoc/org/apache/kafka/streams/KafkaStreams.State.html
    
    I also fixed the diagram in StreamThread.java however currently no javadoc is generated for that one (since it's internal)
    
    enothereska please have a look
    
    Author: Clemens Valiente 
    
    Reviewers: Eno Thereska, Matthias J. Sax, Guozhang Wang
    
    Closes #2621 from cvaliente/KAFKA-4800-ASCII-diagrams
    ---
     .../apache/kafka/streams/KafkaStreams.java    | 42 +++++++-------
     .../processor/internals/StreamThread.java     | 55 ++++++++++---------
     2 files changed, 50 insertions(+), 47 deletions(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    index 0cdeb73b55a05..2810ff884bc4e 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    @@ -142,30 +142,32 @@ public class KafkaStreams {
          * Note this instance will be in "Rebalancing" state if any of its threads is rebalancing
          * The expected state transition with the following defined states is:
          *
    -     *                 +-----------+
    -     *         +<------|Created    |
    -     *         |       +-----+-----+
    -     *         |             |   +--+
    -     *         |             v   |  |
    -     *         |       +-----+---v--+--+
    -     *         +<----- | Rebalancing   |<--------+
    -     *         |       +-----+---------+         ^
    -     *         |                 +--+            |
    -     *         |                 |  |            |
    -     *         |       +-----+---v--+-----+      |
    -     *         +------>|Running           |------+
    -     *         |       +-----+------------+
    +     * 
    +     *                 +--------------+
    +     *         +<----- | Created      |
    +     *         |       +-----+--------+
          *         |             |
          *         |             v
    -     *         |     +-------+--------+
    -     *         +---->|Pending         |
    -     *               |Shutdown        |
    -     *               +-------+--------+
    +     *         |       +-----+--------+
    +     *         +<----- | Rebalancing  | <----+
    +     *         |       +--------------+      |
    +     *         |                             |
    +     *         |                             |
    +     *         |       +--------------+      |
    +     *         +-----> | Running      | ---->+
    +     *         |       +-----+--------+
    +     *         |             |
    +     *         |             v
    +     *         |       +-----+--------+
    +     *         +-----> | Pending      |
    +     *                 | Shutdown     |
    +     *                 +-----+--------+
          *                       |
          *                       v
    -     *                 +-----+-----+
    -     *                 |Not Running|
    -     *                 +-----------+
    +     *                 +-----+--------+
    +     *                 | Not Running  |
    +     *                 +--------------+
    +     * 
    */ public enum State { CREATED(1, 2, 3), RUNNING(2, 3), REBALANCING(1, 2, 3), PENDING_SHUTDOWN(4), NOT_RUNNING; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 033dc7321ff1e..d447824492253 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -74,33 +74,34 @@ public class StreamThread extends Thread { * A thread must only be in one state at a time * The expected state transitions with the following defined states is: * - * +-----------+ - * |Not Running|<---------------+ - * +-----+-----+ | - * | | - * v | - * +-----+-----+ | - * +-----| Running |<------------+ | - * | +-----+-----+ | | - * | | | | - * | v | | - * | +-----+------------+ | | - * <---- |Partitions | | | - * | |Revoked | | | - * | +-----+------------+ | | - * | | | | - * | v | | - * | +-----+------------+ | | - * | |Assigning | | | - * | |Partitions |------+ | - * | +-----+------------+ | - * | | | - * | | | - * | +------v---------+ | - * +--->|Pending |------------+ - * |Shutdown | - * +-----+----------+ - * + *
    +     *                +-------------+
    +     *                | Not Running | <-------+
    +     *                +-----+-------+         |
    +     *                      |                 |
    +     *                      v                 |
    +     *                +-----+-------+         |
    +     *          +<--- | Running     | <----+  |
    +     *          |     +-----+-------+      |  |
    +     *          |           |              |  |
    +     *          |           v              |  |
    +     *          |     +-----+-------+      |  |
    +     *          +<--- | Partitions  |      |  |
    +     *          |     | Revoked     |      |  |
    +     *          |     +-----+-------+      |  |
    +     *          |           |              |  |
    +     *          |           v              |  |
    +     *          |     +-----+-------+      |  |
    +     *          |     | Assigning   |      |  |
    +     *          |     | Partitions  | ---->+  |
    +     *          |     +-----+-------+         |
    +     *          |           |                 |
    +     *          |           v                 |
    +     *          |     +-----+-------+         |
    +     *          +---> | Pending     | ------->+
    +     *                | Shutdown    |
    +     *                +-------------+
    +     * 
    */ public enum State { NOT_RUNNING(1), RUNNING(1, 2, 4), PARTITIONS_REVOKED(3, 4), ASSIGNING_PARTITIONS(1, 4), PENDING_SHUTDOWN(0); From a3c45b0c9263ca4481b1af7af3f7d8ae8cb5d5cd Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Thu, 2 Mar 2017 17:49:01 -0800 Subject: [PATCH 057/101] KAFKA-4631; Request metadata in consumer if topic/partitions unavailable If leader node of one more more partitions in a consumer subscription are temporarily unavailable, request metadata refresh so that partitions skipped for assignment dont have to wait for metadata expiry before reassignment. Metadata refresh is also requested if a subscribe topic or assigned partition doesn't exist. Author: Rajini Sivaram Reviewers: Vahid Hashemian , Ismael Juma , Jason Gustafson Closes #2622 from rajinisivaram/KAFKA-4631 --- .../org/apache/kafka/clients/Metadata.java | 18 +++- .../apache/kafka/clients/NetworkClient.java | 2 +- .../kafka/clients/consumer/KafkaConsumer.java | 2 +- .../internals/ConsumerCoordinator.java | 6 +- .../kafka/clients/producer/KafkaProducer.java | 2 +- .../common/requests/MetadataResponse.java | 24 +++++ .../apache/kafka/clients/MetadataTest.java | 66 ++++++------- .../org/apache/kafka/clients/MockClient.java | 33 +++++-- .../kafka/clients/NetworkClientTest.java | 2 +- .../clients/consumer/KafkaConsumerTest.java | 38 ++++---- .../internals/AbstractCoordinatorTest.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 94 +++++++++++++++---- .../consumer/internals/FetcherTest.java | 4 +- .../producer/internals/SenderTest.java | 14 +-- .../distributed/WorkerGroupMember.java | 3 +- .../distributed/WorkerCoordinatorTest.java | 2 +- .../main/scala/kafka/admin/AdminClient.scala | 2 +- .../kafka/api/ConsumerBounceTest.scala | 59 +++++++++++- .../internals/StreamsKafkaClient.java | 4 +- 19 files changed, 270 insertions(+), 107 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 87e5862727758..5bfdb642f9c6a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -199,8 +199,13 @@ public synchronized boolean containsTopic(String topic) { /** * Updates the cluster metadata. If topic expiry is enabled, expiry time * is set for topics if required and expired topics are removed from the metadata. + * + * @param cluster the cluster containing metadata for topics with valid metadata + * @param unavailableTopics topics which are non-existent or have one or more partitions whose + * leader is not known + * @param now current time in milliseconds */ - public synchronized void update(Cluster cluster, long now) { + public synchronized void update(Cluster cluster, Set unavailableTopics, long now) { Objects.requireNonNull(cluster, "cluster should not be null"); this.needUpdate = false; @@ -223,7 +228,7 @@ else if (expireMs <= now) { } for (Listener listener: listeners) - listener.onMetadataUpdate(cluster); + listener.onMetadataUpdate(cluster, unavailableTopics); String previousClusterId = cluster.clusterResource().clusterId(); @@ -306,7 +311,14 @@ public synchronized void removeListener(Listener listener) { * MetadataUpdate Listener */ public interface Listener { - void onMetadataUpdate(Cluster cluster); + /** + * Callback invoked on metadata update. + * + * @param cluster the cluster containing metadata for topics with valid metadata + * @param unavailableTopics topics which are non-existent or have one or more partitions whose + * leader is not known + */ + void onMetadataUpdate(Cluster cluster, Set unavailableTopics); } private synchronized void requestUpdateForNewTopics() { diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 1374a776dadb4..8d3adda0b386a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -715,7 +715,7 @@ public void handleCompletedMetadataResponse(RequestHeader requestHeader, long no // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being // created which means we will get errors and no nodes until it exists if (cluster.nodes().size() > 0) { - this.metadata.update(cluster, now); + this.metadata.update(cluster, response.unavailableTopics(), now); } else { log.trace("Ignoring empty metadata response with correlation id {}.", requestHeader.correlationId()); this.metadata.failedUpdate(now); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 1e3749757595c..b4514c55d370a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -656,7 +656,7 @@ private KafkaConsumer(ConsumerConfig config, ClusterResourceListeners clusterResourceListeners = configureClusterResourceListeners(keyDeserializer, valueDeserializer, reporters, interceptorList); this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG), false, clusterResourceListeners); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), 0); + this.metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), 0); String metricGrpPrefix = "consumer"; ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config); NetworkClient netClient = new NetworkClient( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index dad59ea92e1e5..ba19146ae0436 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -53,6 +53,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -172,7 +173,7 @@ public void updatePatternSubscription(Cluster cluster) { private void addMetadataListener() { this.metadata.addListener(new Metadata.Listener() { @Override - public void onMetadataUpdate(Cluster cluster) { + public void onMetadataUpdate(Cluster cluster, Set unavailableTopics) { // if we encounter any unauthorized topics, raise an exception to the user if (!cluster.unauthorizedTopics().isEmpty()) throw new TopicAuthorizationException(new HashSet<>(cluster.unauthorizedTopics())); @@ -186,6 +187,9 @@ public void onMetadataUpdate(Cluster cluster) { if (!snapshot.equals(metadataSnapshot)) metadataSnapshot = snapshot; } + + if (!Collections.disjoint(metadata.topics(), unavailableTopics)) + metadata.requestUpdate(); } }); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 5f83e40ab9ba7..33da0c4e03d7a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -305,7 +305,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial time); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); + this.metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), time.milliseconds()); ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config); NetworkClient client = new NetworkClient( new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", channelBuilder), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 02dbe392cb66c..51aaa239f8ad0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; @@ -203,6 +204,29 @@ public Set topicsByError(Errors error) { return errorTopics; } + /** + * Returns the set of topics with an error indicating invalid metadata + * and topics with any partition whose error indicates invalid metadata. + * This includes all non-existent topics specified in the metadata request + * and any topic returned with one or more partitions whose leader is not known. + */ + public Set unavailableTopics() { + Set invalidMetadataTopics = new HashSet<>(); + for (TopicMetadata topicMetadata : this.topicMetadata) { + if (topicMetadata.error.exception() instanceof InvalidMetadataException) + invalidMetadataTopics.add(topicMetadata.topic); + else { + for (PartitionMetadata partitionMetadata : topicMetadata.partitionMetadata) { + if (partitionMetadata.error.exception() instanceof InvalidMetadataException) { + invalidMetadataTopics.add(topicMetadata.topic); + break; + } + } + } + } + return invalidMetadataTopics; + } + /** * Get a snapshot of the cluster metadata from this response * @return the cluster snapshot diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index 084ccd8650d87..0c87fc75d8dce 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -56,7 +56,7 @@ public void tearDown() { @Test public void testMetadata() throws Exception { long time = 0; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); metadata.requestUpdate(); assertFalse("Still no updated needed due to backoff", metadata.timeToNextUpdate(time) == 0); @@ -71,7 +71,7 @@ public void testMetadata() throws Exception { // This simulates the metadata update sequence in KafkaProducer while (t1.isAlive() || t2.isAlive()) { if (metadata.timeToNextUpdate(time) == 0) { - metadata.update(TestUtils.singletonCluster(topic, 1), time); + metadata.update(TestUtils.singletonCluster(topic, 1), Collections.emptySet(), time); time += refreshBackoffMs; } Thread.sleep(1); @@ -101,7 +101,7 @@ private static void checkTimeToNextUpdate(long refreshBackoffMs, long metadataEx assertEquals(0, metadata.timeToNextUpdate(now)); // lastSuccessfulRefreshMs updated to now. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); // The last update was successful so the remaining time to expire the current metadata should be returned. assertEquals(largerOfBackoffAndExpire, metadata.timeToNextUpdate(now)); @@ -112,7 +112,7 @@ private static void checkTimeToNextUpdate(long refreshBackoffMs, long metadataEx assertEquals(refreshBackoffMs, metadata.timeToNextUpdate(now)); // Reset needUpdate to false. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); assertEquals(largerOfBackoffAndExpire, metadata.timeToNextUpdate(now)); // Both metadataExpireMs and refreshBackoffMs elapsed. @@ -156,13 +156,13 @@ public void testTimeToNextUpdate_OverwriteBackoff() { long now = 10000; // New topic added to fetch set and update requested. It should allow immediate update. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.add("new-topic"); assertEquals(0, metadata.timeToNextUpdate(now)); // Even though setTopics called, immediate update isn't necessary if the new topic set isn't // containing a new topic, - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.setTopics(metadata.topics()); assertEquals(metadataExpireMs, metadata.timeToNextUpdate(now)); @@ -171,12 +171,12 @@ public void testTimeToNextUpdate_OverwriteBackoff() { assertEquals(0, metadata.timeToNextUpdate(now)); // If metadata requested for all topics it should allow immediate update. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.needMetadataForAllTopics(true); assertEquals(0, metadata.timeToNextUpdate(now)); // However if metadata is already capable to serve all topics it shouldn't override backoff. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.needMetadataForAllTopics(true); assertEquals(metadataExpireMs, metadata.timeToNextUpdate(now)); } @@ -191,7 +191,7 @@ public void testTimeToNextUpdate_OverwriteBackoff() { @Test public void testMetadataUpdateWaitTime() throws Exception { long time = 0; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); // first try with a max wait time of 0 and ensure that this returns back without waiting forever try { @@ -213,7 +213,7 @@ public void testMetadataUpdateWaitTime() throws Exception { @Test public void testFailedUpdate() { long time = 100; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertEquals(100, metadata.timeToNextUpdate(1000)); metadata.failedUpdate(1100); @@ -222,14 +222,14 @@ public void testFailedUpdate() { assertEquals(100, metadata.lastSuccessfulUpdate()); metadata.needMetadataForAllTopics(true); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertEquals(100, metadata.timeToNextUpdate(1000)); } @Test public void testUpdateWithNeedMetadataForAllTopics() { long time = 0; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); metadata.needMetadataForAllTopics(true); final List expectedTopics = Collections.singletonList("topic"); @@ -241,7 +241,7 @@ public void testUpdateWithNeedMetadataForAllTopics() { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertArrayEquals("Metadata got updated with wrong set of topics.", expectedTopics.toArray(), metadata.topics().toArray()); @@ -259,7 +259,7 @@ public void testClusterListenerGetsNotifiedOfUpdate() { String hostName = "www.example.com"; Cluster cluster = Cluster.bootstrap(Arrays.asList(new InetSocketAddress(hostName, 9002))); - metadata.update(cluster, time); + metadata.update(cluster, Collections.emptySet(), time); assertFalse("ClusterResourceListener should not called when metadata is updated with bootstrap Cluster", MockClusterResourceListener.IS_ON_UPDATE_CALLED.get()); @@ -271,7 +271,7 @@ public void testClusterListenerGetsNotifiedOfUpdate() { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertEquals("MockClusterResourceListener did not get cluster metadata correctly", "dummy", mockClusterListener.clusterResource().clusterId()); @@ -283,10 +283,10 @@ public void testClusterListenerGetsNotifiedOfUpdate() { public void testListenerGetsNotifiedOfUpdate() { long time = 0; final Set topics = new HashSet<>(); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); metadata.addListener(new Metadata.Listener() { @Override - public void onMetadataUpdate(Cluster cluster) { + public void onMetadataUpdate(Cluster cluster, Set unavailableTopics) { topics.clear(); topics.addAll(cluster.topics()); } @@ -300,7 +300,7 @@ public void onMetadataUpdate(Cluster cluster) { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertEquals("Listener did not update topics list correctly", new HashSet<>(Arrays.asList("topic", "topic1")), topics); @@ -310,10 +310,10 @@ public void onMetadataUpdate(Cluster cluster) { public void testListenerCanUnregister() { long time = 0; final Set topics = new HashSet<>(); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); final Metadata.Listener listener = new Metadata.Listener() { @Override - public void onMetadataUpdate(Cluster cluster) { + public void onMetadataUpdate(Cluster cluster, Set unavailableTopics) { topics.clear(); topics.addAll(cluster.topics()); } @@ -328,7 +328,7 @@ public void onMetadataUpdate(Cluster cluster) { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); metadata.removeListener(listener); @@ -340,7 +340,7 @@ public void onMetadataUpdate(Cluster cluster) { new PartitionInfo("topic3", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertEquals("Listener did not update topics list correctly", new HashSet<>(Arrays.asList("topic", "topic1")), topics); @@ -353,17 +353,17 @@ public void testTopicExpiry() throws Exception { // Test that topic is expired if not used within the expiry interval long time = 0; metadata.add("topic1"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); time += Metadata.TOPIC_EXPIRY_MS; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("Unused topic not expired", metadata.containsTopic("topic1")); // Test that topic is not expired if used within the expiry interval metadata.add("topic2"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); for (int i = 0; i < 3; i++) { time += Metadata.TOPIC_EXPIRY_MS / 2; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Topic expired even though in use", metadata.containsTopic("topic2")); metadata.add("topic2"); } @@ -372,9 +372,9 @@ public void testTopicExpiry() throws Exception { HashSet topics = new HashSet<>(); topics.add("topic4"); metadata.setTopics(topics); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); time += Metadata.TOPIC_EXPIRY_MS; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("Unused topic not expired", metadata.containsTopic("topic4")); } @@ -385,17 +385,17 @@ public void testNonExpiringMetadata() throws Exception { // Test that topic is not expired if not used within the expiry interval long time = 0; metadata.add("topic1"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); time += Metadata.TOPIC_EXPIRY_MS; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Unused topic expired when expiry disabled", metadata.containsTopic("topic1")); // Test that topic is not expired if used within the expiry interval metadata.add("topic2"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); for (int i = 0; i < 3; i++) { time += Metadata.TOPIC_EXPIRY_MS / 2; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Topic expired even though in use", metadata.containsTopic("topic2")); metadata.add("topic2"); } @@ -405,7 +405,7 @@ public void testNonExpiringMetadata() throws Exception { topics.add("topic4"); metadata.setTopics(topics); time += metadataExpireMs * 2; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Unused topic expired when expiry disabled", metadata.containsTopic("topic4")); } diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 94a7050c0aeb2..f97e407ed8d44 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -26,6 +26,7 @@ import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -63,6 +64,7 @@ public FutureResponse(AbstractResponse responseBody, boolean disconnected, Reque private final Time time; private final Metadata metadata; + private Set unavailableTopics; private int correlation = 0; private Node node = null; private final Set ready = new HashSet<>(); @@ -72,17 +74,17 @@ public FutureResponse(AbstractResponse responseBody, boolean disconnected, Reque // Use concurrent queue for responses so that responses may be updated during poll() from a different thread. private final Queue responses = new ConcurrentLinkedDeque<>(); private final Queue futureResponses = new ArrayDeque<>(); - private final Queue metadataUpdates = new ArrayDeque<>(); + private final Queue metadataUpdates = new ArrayDeque<>(); private volatile NodeApiVersions nodeApiVersions = NodeApiVersions.create(); public MockClient(Time time) { - this.time = time; - this.metadata = null; + this(time, null); } public MockClient(Time time, Metadata metadata) { this.time = time; this.metadata = metadata; + this.unavailableTopics = Collections.emptySet(); } @Override @@ -166,11 +168,13 @@ public List poll(long timeoutMs, long now) { List copy = new ArrayList<>(this.responses); if (metadata != null && metadata.updateRequested()) { - Cluster cluster = metadataUpdates.poll(); - if (cluster == null) - metadata.update(metadata.fetch(), time.milliseconds()); - else - metadata.update(cluster, time.milliseconds()); + MetadataUpdate metadataUpdate = metadataUpdates.poll(); + if (metadataUpdate == null) + metadata.update(metadata.fetch(), this.unavailableTopics, time.milliseconds()); + else { + this.unavailableTopics = metadataUpdate.unavailableTopics; + metadata.update(metadataUpdate.cluster, metadataUpdate.unavailableTopics, time.milliseconds()); + } } while (!this.responses.isEmpty()) { @@ -278,8 +282,8 @@ public void reset() { metadataUpdates.clear(); } - public void prepareMetadataUpdate(Cluster cluster) { - metadataUpdates.add(cluster); + public void prepareMetadataUpdate(Cluster cluster, Set unavailableTopics) { + metadataUpdates.add(new MetadataUpdate(cluster, unavailableTopics)); } public void setNode(Node node) { @@ -340,4 +344,13 @@ public interface RequestMatcher { public void setNodeApiVersions(NodeApiVersions nodeApiVersions) { this.nodeApiVersions = nodeApiVersions; } + + private static class MetadataUpdate { + final Cluster cluster; + final Set unavailableTopics; + MetadataUpdate(Cluster cluster, Set unavailableTopics) { + this.cluster = cluster; + this.unavailableTopics = unavailableTopics; + } + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 2fba771b9b2f5..d22c04afe263d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -76,7 +76,7 @@ private NetworkClient createNetworkClientWithNoVersionDiscovery() { @Before public void setup() { - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); } @Test(expected = IllegalStateException.class) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 864700b0ae697..d3fc54744d952 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -364,7 +364,7 @@ public void verifyHeartbeatSent() throws Exception { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -405,7 +405,7 @@ public void verifyHeartbeatSentWhenFetchedDataReady() throws Exception { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -446,7 +446,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -482,7 +482,7 @@ public void testCommitsFetchedDuringAssign() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -530,7 +530,7 @@ public void testAutoCommitSentBeforePositionUpdate() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -591,7 +591,7 @@ public void testRegexSubscription() { consumer.subscribe(Pattern.compile(topic), getConsumerRebalanceListener(consumer)); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); consumer.poll(0); assertEquals(singleton(topic), consumer.subscription()); @@ -622,7 +622,7 @@ public void testChangingRegexSubscription() { MockClient client = new MockClient(time, metadata); client.setNode(node); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, false, autoCommitIntervalMs); @@ -630,14 +630,14 @@ public void testChangingRegexSubscription() { Node coordinator = prepareRebalance(client, node, singleton(topic), assignor, singletonList(tp0), null); consumer.subscribe(Pattern.compile(topic), getConsumerRebalanceListener(consumer)); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); consumer.poll(0); assertEquals(singleton(topic), consumer.subscription()); consumer.subscribe(Pattern.compile(otherTopic), getConsumerRebalanceListener(consumer)); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); prepareRebalance(client, node, singleton(otherTopic), assignor, singletonList(otherTopicPartition), coordinator); consumer.poll(0); @@ -660,7 +660,7 @@ public void testWakeupWithFetchDataAvailable() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -705,7 +705,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() throws Exception { final Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); final MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -745,7 +745,7 @@ public void fetchResponseWithUnexpectedPartitionIsIgnored() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -793,7 +793,7 @@ public void testSubscriptionChangesWithAutoCommitEnabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -913,7 +913,7 @@ public void testSubscriptionChangesWithAutoCommitDisabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -981,7 +981,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -1046,7 +1046,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -1107,7 +1107,7 @@ public void testOffsetOfPausedPartitions() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -1226,7 +1226,7 @@ private void consumerCloseTest(final long closeTimeoutMs, Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -1238,7 +1238,7 @@ private void consumerCloseTest(final long closeTimeoutMs, consumer.subscribe(Arrays.asList(topic), getConsumerRebalanceListener(consumer)); Node coordinator = prepareRebalance(client, node, assignor, Arrays.asList(tp0), null); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); // Poll with responses client.prepareResponseFrom(fetchResponse(tp0, 0, 1), node); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 3450f31bb7658..3eb6561b39a0c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -79,7 +79,7 @@ public void setupCoordinator() { Metrics metrics = new Metrics(); Cluster cluster = TestUtils.singletonCluster("topic", 1); - metadata.update(cluster, mockTime.milliseconds()); + metadata.update(cluster, Collections.emptySet(), mockTime.milliseconds()); this.node = cluster.nodes().get(0); mockClient.setNode(node); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index a89b67c00810d..9cce87472d444 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -120,7 +120,7 @@ public void setup() { this.time = new MockTime(); this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); this.metadata = new Metadata(0, Long.MAX_VALUE); - this.metadata.update(cluster, time.milliseconds()); + this.metadata.update(cluster, Collections.emptySet(), time.milliseconds()); this.client = new MockClient(time, metadata); this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); @@ -292,7 +292,7 @@ public void testJoinGroupInvalidGroupId() { // ensure metadata is up-to-date for leader metadata.setTopics(singletonList(topic1)); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(); @@ -310,7 +310,7 @@ public void testNormalJoinGroupLeader() { // ensure metadata is up-to-date for leader metadata.setTopics(singletonList(topic1)); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(); @@ -349,7 +349,7 @@ public void testPatternJoinGroupLeader() { // partially update the metadata with one topic first, // let the leader to refresh metadata during assignment metadata.setTopics(singletonList(topic1)); - metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(); @@ -369,7 +369,7 @@ public boolean matches(AbstractRequest body) { } }, syncGroupResponse(Arrays.asList(t1p, t2p), Errors.NONE)); // expect client to force updating the metadata, if yes gives it both topics - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); coordinator.poll(time.milliseconds()); @@ -389,7 +389,7 @@ public void testMetadataRefreshDuringRebalance() { subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); metadata.needMetadataForAllTopics(true); - metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); assertEquals(singleton(topic1), subscriptions.subscription()); @@ -410,7 +410,7 @@ public boolean matches(AbstractRequest body) { final Map updatedPartitions = new HashMap<>(); for (String topic : updatedSubscription) updatedPartitions.put(topic, 1); - metadata.update(TestUtils.clusterWith(1, updatedPartitions), time.milliseconds()); + metadata.update(TestUtils.clusterWith(1, updatedPartitions), Collections.emptySet(), time.milliseconds()); return true; } }, syncGroupResponse(singletonList(t1p), Errors.NONE)); @@ -453,7 +453,7 @@ public void testWakeupDuringJoin() { // ensure metadata is up-to-date for leader metadata.setTopics(singletonList(topic1)); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(); @@ -524,7 +524,7 @@ public void testPatternJoinGroupFollower() { // partially update the metadata with one topic first, // let the leader to refresh metadata during assignment metadata.setTopics(singletonList(topic1)); - metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(); @@ -541,7 +541,7 @@ public boolean matches(AbstractRequest body) { } }, syncGroupResponse(Arrays.asList(t1p, t2p), Errors.NONE)); // expect client to force updating the metadata, if yes gives it both topics - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); coordinator.joinGroupIfNeeded(); @@ -712,7 +712,7 @@ public void testMetadataChangeTriggersRebalance() { // ensure metadata is up-to-date for leader metadata.setTopics(singletonList(topic1)); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); subscriptions.subscribe(singleton(topic1), rebalanceListener); @@ -731,7 +731,7 @@ public void testMetadataChangeTriggersRebalance() { assertFalse(coordinator.needRejoin()); // a new partition is added to the topic - metadata.update(TestUtils.singletonCluster(topic1, 2), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 2), Collections.emptySet(), time.milliseconds()); // we should detect the change and ask for reassignment assertTrue(coordinator.needRejoin()); @@ -751,7 +751,7 @@ public void testUpdateMetadataDuringRebalance() { metadata.setTopics(topics); // we only have metadata for one topic initially - metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(); @@ -772,7 +772,7 @@ public boolean matches(AbstractRequest body) { Map topicPartitionCounts = new HashMap<>(); topicPartitionCounts.put(topic1, 1); topicPartitionCounts.put(topic2, 1); - metadata.update(TestUtils.singletonCluster(topicPartitionCounts), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topicPartitionCounts), Collections.emptySet(), time.milliseconds()); return true; } return false; @@ -789,12 +789,72 @@ public boolean matches(AbstractRequest body) { assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions()); } + @Test + public void testRebalanceAfterTopicUnavailableWithSubscribe() { + unavailableTopicTest(false, false, Collections.emptySet()); + } + + @Test + public void testRebalanceAfterTopicUnavailableWithPatternSubscribe() { + unavailableTopicTest(true, false, Collections.emptySet()); + } + + @Test + public void testRebalanceAfterNotMatchingTopicUnavailableWithPatternSSubscribe() { + unavailableTopicTest(true, false, Collections.singleton("notmatching")); + } + + @Test + public void testAssignWithTopicUnavailable() { + unavailableTopicTest(true, false, Collections.emptySet()); + } + + private void unavailableTopicTest(boolean patternSubscribe, boolean assign, Set unavailableTopicsInLastMetadata) { + final String consumerId = "consumer"; + + metadata.setTopics(singletonList(topic1)); + client.prepareMetadataUpdate(Cluster.empty(), Collections.singleton("test1")); + + if (assign) + subscriptions.assignFromUser(singleton(t1p)); + else if (patternSubscribe) + subscriptions.subscribe(Pattern.compile("test.*"), rebalanceListener); + else + subscriptions.subscribe(singleton(topic1), rebalanceListener); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(); + + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.>emptyMap()); + + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE)); + client.prepareResponse(syncGroupResponse(Collections.emptyList(), Errors.NONE)); + coordinator.poll(time.milliseconds()); + if (!assign) { + assertFalse(coordinator.needRejoin()); + assertEquals(Collections.emptySet(), rebalanceListener.assigned); + } + assertTrue("Metadata refresh not requested for unavailable partitions", metadata.updateRequested()); + + client.prepareMetadataUpdate(cluster, unavailableTopicsInLastMetadata); + client.poll(0, time.milliseconds()); + client.prepareResponse(joinGroupLeaderResponse(2, consumerId, memberSubscriptions, Errors.NONE)); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); + coordinator.poll(time.milliseconds()); + + assertFalse("Metadata refresh requested unnecessarily", metadata.updateRequested()); + if (!assign) { + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(t1p), rebalanceListener.assigned); + } + } @Test public void testExcludeInternalTopicsConfigOption() { subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); - metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), Collections.emptySet(), time.milliseconds()); assertFalse(subscriptions.subscription().contains(TestUtils.GROUP_METADATA_TOPIC_NAME)); } @@ -804,7 +864,7 @@ public void testIncludeInternalTopicsConfigOption() { coordinator = buildCoordinator(new Metrics(), assignors, false, false); subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); - metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), Collections.emptySet(), time.milliseconds()); assertTrue(subscriptions.subscription().contains(TestUtils.GROUP_METADATA_TOPIC_NAME)); } @@ -1030,7 +1090,7 @@ public void testCommitAfterLeaveGroup() { client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); coordinator.joinGroupIfNeeded(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 10bac59d76f13..3f65caf244d52 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -109,7 +109,7 @@ public class FetcherTest { @Before public void setup() throws Exception { - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.setNode(node); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 1L); @@ -838,7 +838,7 @@ private void testGetOffsetsForTimesWithError(Errors errorForTp0, TopicPartition tp1 = new TopicPartition(topicName, 1); // Ensure metadata has both partition. Cluster cluster = TestUtils.clusterWith(2, topicName, 2); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); // First try should fail due to metadata error. client.prepareResponseFrom(listOffsetResponse(tp0, errorForTp0, offsetForTp0, offsetForTp0), cluster.leaderFor(tp0)); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 2210b6d31f73d..e9a7188739b33 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -88,7 +88,7 @@ public void setup() { time, REQUEST_TIMEOUT); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); } @After @@ -201,7 +201,7 @@ public void testSendInOrder() throws Exception { // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 Cluster cluster1 = TestUtils.clusterWith(2, "test", 2); - metadata.update(cluster1, time.milliseconds()); + metadata.update(cluster1, Collections.emptySet(), time.milliseconds()); // Send the first message. TopicPartition tp2 = new TopicPartition("test", 1); @@ -220,7 +220,7 @@ public void testSendInOrder() throws Exception { // Update metadata before sender receives response from broker 0. Now partition 2 moves to broker 0 Cluster cluster2 = TestUtils.singletonCluster("test", 2); - metadata.update(cluster2, time.milliseconds()); + metadata.update(cluster2, Collections.emptySet(), time.milliseconds()); // Sender should not send the second message to node 0. sender.run(time.milliseconds()); assertEquals(1, client.inFlightRequestCount()); @@ -236,12 +236,12 @@ public void testSendInOrder() throws Exception { @Test public void testMetadataTopicExpiry() throws Exception { long offset = 0; - metadata.update(Cluster.empty(), time.milliseconds()); + metadata.update(Cluster.empty(), Collections.emptySet(), time.milliseconds()); Future future = accumulator.append(tp, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); assertTrue("Topic not added to metadata", metadata.containsTopic(tp.topic())); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); sender.run(time.milliseconds()); // send produce request client.respond(produceResponse(tp, offset++, Errors.NONE, 0)); sender.run(time.milliseconds()); @@ -251,12 +251,12 @@ public void testMetadataTopicExpiry() throws Exception { assertTrue("Topic not retained in metadata list", metadata.containsTopic(tp.topic())); time.sleep(Metadata.TOPIC_EXPIRY_MS); - metadata.update(Cluster.empty(), time.milliseconds()); + metadata.update(Cluster.empty(), Collections.emptySet(), time.milliseconds()); assertFalse("Unused topic has not been expired", metadata.containsTopic(tp.topic())); future = accumulator.append(tp, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); assertTrue("Topic not added to metadata", metadata.containsTopic(tp.topic())); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); sender.run(time.milliseconds()); // send produce request client.respond(produceResponse(tp, offset++, Errors.NONE, 0)); sender.run(time.milliseconds()); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 88c9e35845641..975d4238bb959 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -37,6 +37,7 @@ import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -87,7 +88,7 @@ public WorkerGroupMember(DistributedConfig config, this.retryBackoffMs = config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG); this.metadata = new Metadata(retryBackoffMs, config.getLong(CommonClientConfigs.METADATA_MAX_AGE_CONFIG)); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), 0); + this.metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), 0); String metricGrpPrefix = "connect"; ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config); NetworkClient netClient = new NetworkClient( diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index 045b5bbcee096..977cc21ff0bf9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -91,7 +91,7 @@ public void setup() { this.time = new MockTime(); this.client = new MockClient(time); this.metadata = new Metadata(0, Long.MAX_VALUE); - this.metadata.update(cluster, time.milliseconds()); + this.metadata.update(cluster, Collections.emptySet(), time.milliseconds()); this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 49c328e3e7ef1..7cfc91a6525aa 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -243,7 +243,7 @@ object AdminClient { val brokerUrls = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) val brokerAddresses = ClientUtils.parseAndValidateAddresses(brokerUrls) val bootstrapCluster = Cluster.bootstrap(brokerAddresses) - metadata.update(bootstrapCluster, 0) + metadata.update(bootstrapCluster, Collections.emptySet(), 0) val selector = new Selector( DefaultConnectionMaxIdleMs, diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 2f37b5ba63ab4..f0e0c9e9bac0d 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -43,13 +43,14 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { // Time to process commit and leave group requests in tests when brokers are available val gracefulCloseTimeMs = 1000 - val executor = Executors.newFixedThreadPool(2) + val executor = Executors.newScheduledThreadPool(2) // configure the servers and clients this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") this.serverConfig.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "10") // set small enough session timeout + this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) @@ -161,6 +162,52 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { } } + @Test + def testSubscribeWhenTopicUnavailable() { + val numRecords = 1000 + val newtopic = "newtopic" + + val consumer = this.consumers.head + consumer.subscribe(Collections.singleton(newtopic)) + executor.schedule(new Runnable { + def run() = TestUtils.createTopic(zkUtils, newtopic, serverCount, serverCount, servers) + }, 2, TimeUnit.SECONDS) + consumer.poll(0) + + def sendRecords(numRecords: Int, topic: String = this.topic) { + var remainingRecords = numRecords + val endTimeMs = System.currentTimeMillis + 20000 + while (remainingRecords > 0 && System.currentTimeMillis < endTimeMs) { + val futures = (0 until remainingRecords).map { i => + this.producers.head.send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + } + futures.map { future => + try { + future.get + remainingRecords -= 1 + } catch { + case _: Exception => + } + } + } + assertEquals(0, remainingRecords) + } + + sendRecords(numRecords, newtopic) + receiveRecords(consumer, numRecords, newtopic, 10000) + + servers.foreach(server => killBroker(server.config.brokerId)) + Thread.sleep(500) + restartDeadBrokers() + + val future = executor.submit(new Runnable { + def run() = receiveRecords(consumer, numRecords, newtopic, 10000) + }) + sendRecords(numRecords, newtopic) + future.get + } + + @Test def testClose() { val numRecords = 10 @@ -312,10 +359,12 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { consumer } - private def receiveRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) { - var received = 0 - while (received < numRecords) + private def receiveRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int, topic: String = this.topic, timeoutMs: Long = 60000) { + var received = 0L + val endTimeMs = System.currentTimeMillis + timeoutMs + while (received < numRecords && System.currentTimeMillis < endTimeMs) received += consumer.poll(1000).count() + assertEquals(numRecords, received) } private def submitCloseAndValidate(consumer: KafkaConsumer[Array[Byte], Array[Byte]], @@ -373,7 +422,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { } } - private def sendRecords(numRecords: Int) { + private def sendRecords(numRecords: Int, topic: String = this.topic) { val futures = (0 until numRecords).map { i => this.producers.head.send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 8ee5fe03d9201..4ca0b4d51ae79 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -98,7 +98,7 @@ public StreamsKafkaClient(final Config streamsConfig) { streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG) ); final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); + metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), time.milliseconds()); final MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(streamsConfig.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) @@ -226,7 +226,7 @@ private String getAnyReadyBrokerId() { streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - metadata.update(Cluster.bootstrap(addresses), Time.SYSTEM.milliseconds()); + metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), Time.SYSTEM.milliseconds()); final List nodes = metadata.fetch().nodes(); return ensureOneNodeIsReady(nodes); From 573a6f39863061a6f38a0aca35f11470c3e8538e Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Fri, 3 Mar 2017 11:22:42 -0800 Subject: [PATCH 058/101] KAFKA-2857; Retry querying the consumer group while initializing This applies to new-consumer based groups and would avoid scenarios in which user issues a `--describe` query while the group is initializing. Example: The following could occur for a newly created group. ``` kafkakafka:~/workspace/kafka$ bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group g Note: This will only show information about consumers that use the Java consumer API (non-ZooKeeper-based consumers). Error: Executing consumer group command failed due to The group coordinator is not available. ``` With this PR the group is queried repeatedly at specific intervals within a preset (and configurable) timeout `group-init-timeout` to circumvent unfortunate situations like above. Author: Vahid Hashemian Reviewers: Jason Gustafson Closes #2538 from vahidhashemian/KAFKA-2857 --- .../main/scala/kafka/admin/AdminClient.scala | 48 +++++-- .../kafka/admin/ConsumerGroupCommand.scala | 17 ++- .../scala/kafka/tools/StreamsResetter.java | 2 +- .../admin/DescribeConsumerGroupTest.scala | 133 +++++++----------- .../integration/ResetIntegrationTest.java | 2 +- 5 files changed, 110 insertions(+), 92 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 7cfc91a6525aa..4b284608a258a 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -16,18 +16,21 @@ import java.nio.ByteBuffer import java.util.{Collections, Properties} import java.util.concurrent.atomic.AtomicInteger -import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion import kafka.common.KafkaException import kafka.coordinator.GroupOverview import kafka.utils.Logging + import org.apache.kafka.clients._ import org.apache.kafka.clients.consumer.internals.{ConsumerNetworkClient, ConsumerProtocol, RequestFuture} import org.apache.kafka.common.config.ConfigDef.{Importance, Type} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} +import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.Selector import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests._ +import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion +import org.apache.kafka.common.requests.DescribeGroupsResponse.GroupMetadata import org.apache.kafka.common.requests.OffsetFetchResponse import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{Cluster, Node, TopicPartition} @@ -37,6 +40,7 @@ import scala.util.Try class AdminClient(val time: Time, val requestTimeoutMs: Int, + val retryBackoffMs: Int, val client: ConsumerNetworkClient, val bootstrapBrokers: List[Node]) extends Logging { @@ -66,9 +70,19 @@ class AdminClient(val time: Time, throw new RuntimeException(s"Request $api failed on brokers $bootstrapBrokers") } - def findCoordinator(groupId: String): Node = { + def findCoordinator(groupId: String, timeoutMs: Long = 0): Node = { + val startTime = time.milliseconds val requestBuilder = new GroupCoordinatorRequest.Builder(groupId) - val response = sendAnyNode(ApiKeys.GROUP_COORDINATOR, requestBuilder).asInstanceOf[GroupCoordinatorResponse] + var response = sendAnyNode(ApiKeys.GROUP_COORDINATOR, requestBuilder).asInstanceOf[GroupCoordinatorResponse] + + while (response.error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE && time.milliseconds - startTime < timeoutMs) { + Thread.sleep(retryBackoffMs) + response = sendAnyNode(ApiKeys.GROUP_COORDINATOR, requestBuilder).asInstanceOf[GroupCoordinatorResponse] + } + + if (response.error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE) + throw new TimeoutException("The consumer group command timed out while waiting for group to initialize: ", response.error.exception) + response.error.maybeThrow() response.node } @@ -165,18 +179,34 @@ class AdminClient(val time: Time, consumers: Option[List[ConsumerSummary]], coordinator: Node) - def describeConsumerGroup(groupId: String): ConsumerGroupSummary = { - val coordinator = findCoordinator(groupId) + def describeConsumerGroupHandler(coordinator: Node, groupId: String): GroupMetadata = { val responseBody = send(coordinator, ApiKeys.DESCRIBE_GROUPS, new DescribeGroupsRequest.Builder(Collections.singletonList(groupId))) val response = responseBody.asInstanceOf[DescribeGroupsResponse] val metadata = response.groups.get(groupId) if (metadata == null) throw new KafkaException(s"Response from broker contained no metadata for group $groupId") - if (metadata.state != "Dead" && metadata.state != "Empty" && metadata.protocolType != ConsumerProtocol.PROTOCOL_TYPE) - throw new IllegalArgumentException(s"Consumer Group $groupId with protocol type '${metadata.protocolType}' is not a valid consumer group") + metadata + } + + def describeConsumerGroup(groupId: String, timeoutMs: Long = 0): ConsumerGroupSummary = { + + def isValidConsumerGroupResponse(metadata: DescribeGroupsResponse.GroupMetadata): Boolean = + metadata.error == Errors.NONE && (metadata.state == "Dead" || metadata.state == "Empty" || metadata.protocolType == ConsumerProtocol.PROTOCOL_TYPE) + + val startTime = time.milliseconds + val coordinator = findCoordinator(groupId, timeoutMs) + var metadata = describeConsumerGroupHandler(coordinator, groupId) + + while (!isValidConsumerGroupResponse(metadata) && time.milliseconds - startTime < timeoutMs) { + debug(s"The consumer group response for group '$groupId' is invalid. Retrying the request as the group is initializing ...") + Thread.sleep(retryBackoffMs) + metadata = describeConsumerGroupHandler(coordinator, groupId) + } + + if (!isValidConsumerGroupResponse(metadata)) + throw new TimeoutException("The consumer group command timed out while waiting for group to initialize") - metadata.error.maybeThrow() val consumers = metadata.members.asScala.map { consumer => ConsumerSummary(consumer.memberId, consumer.clientId, consumer.clientHost, metadata.state match { case "Stable" => @@ -204,6 +234,7 @@ object AdminClient { val DefaultSendBufferBytes = 128 * 1024 val DefaultReceiveBufferBytes = 32 * 1024 val DefaultRetryBackoffMs = 100 + val AdminClientIdSequence = new AtomicInteger(1) val AdminConfigDef = { val config = new ConfigDef() @@ -274,6 +305,7 @@ object AdminClient { new AdminClient( time, DefaultRequestTimeoutMs, + DefaultRetryBackoffMs, highLevelClient, bootstrapCluster.nodes.asScala.toList) } diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 11f4f89d91585..caad62a837204 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -396,7 +396,7 @@ object ConsumerGroupCommand extends Logging { } protected def collectGroupAssignment(group: String): (Option[String], Option[Seq[PartitionAssignmentState]]) = { - val consumerGroupSummary = adminClient.describeConsumerGroup(group) + val consumerGroupSummary = adminClient.describeConsumerGroup(group, opts.options.valueOf(opts.timeoutMsOpt)) (Some(consumerGroupSummary.state), consumerGroupSummary.consumers match { case None => @@ -502,7 +502,11 @@ object ConsumerGroupCommand extends Logging { "for every consumer group. For instance --topic t1" + nl + "WARNING: Group deletion only works for old ZK-based consumer groups, and one has to use it carefully to only delete groups that are not active." val NewConsumerDoc = "Use new consumer. This is the default." + val TimeoutMsDoc = "The timeout that can be set for some use cases. For example, it can be used when describing the group " + + "to specify the maximum amount of time in milliseconds to wait before the group stabilizes (when the group is just created, " + + "or is going through some changes)." val CommandConfigDoc = "Property file containing configs to be passed to Admin Client and Consumer." + val parser = new OptionParser val zkConnectOpt = parser.accepts("zookeeper", ZkConnectDoc) .withRequiredArg @@ -524,6 +528,11 @@ object ConsumerGroupCommand extends Logging { val describeOpt = parser.accepts("describe", DescribeDoc) val deleteOpt = parser.accepts("delete", DeleteDoc) val newConsumerOpt = parser.accepts("new-consumer", NewConsumerDoc) + val timeoutMsOpt = parser.accepts("timeout", TimeoutMsDoc) + .withRequiredArg + .describedAs("timeout (ms)") + .ofType(classOf[Long]) + .defaultsTo(5000) val commandConfigOpt = parser.accepts("command-config", CommandConfigDoc) .withRequiredArg .describedAs("command config property file") @@ -531,11 +540,15 @@ object ConsumerGroupCommand extends Logging { val options = parser.parse(args : _*) val useOldConsumer = options.has(zkConnectOpt) + val describeOptPresent = options.has(describeOpt) val allConsumerGroupLevelOpts: Set[OptionSpec[_]] = Set(listOpt, describeOpt, deleteOpt) def checkArgs() { // check required args + if (options.has(timeoutMsOpt) && (!describeOptPresent || useOldConsumer)) + debug(s"Option '$timeoutMsOpt' is applicable only when both '$bootstrapServerOpt' and '$describeOpt' are used.") + if (useOldConsumer) { if (options.has(bootstrapServerOpt)) CommandLineUtils.printUsageAndDie(parser, s"Option '$bootstrapServerOpt' is not valid with '$zkConnectOpt'.") @@ -550,7 +563,7 @@ object ConsumerGroupCommand extends Logging { "committed offset for that group expires.") } - if (options.has(describeOpt)) + if (describeOptPresent) CommandLineUtils.checkRequiredArgs(parser, options, groupOpt) if (options.has(deleteOpt) && !options.has(groupOpt) && !options.has(topicOpt)) CommandLineUtils.printUsageAndDie(parser, "Option %s either takes %s, %s, or both".format(deleteOpt, groupOpt, topicOpt)) diff --git a/core/src/main/scala/kafka/tools/StreamsResetter.java b/core/src/main/scala/kafka/tools/StreamsResetter.java index a61c092bf8f95..83166cddb6fb5 100644 --- a/core/src/main/scala/kafka/tools/StreamsResetter.java +++ b/core/src/main/scala/kafka/tools/StreamsResetter.java @@ -91,7 +91,7 @@ public int run(final String[] args, final Properties config) { adminClient = AdminClient.createSimplePlaintext(options.valueOf(bootstrapServerOption)); final String groupId = options.valueOf(applicationIdOption); - if (!adminClient.describeConsumerGroup(groupId).consumers().get().isEmpty()) { + if (!adminClient.describeConsumerGroup(groupId, 0).consumers().get().isEmpty()) { throw new IllegalStateException("Consumer group '" + groupId + "' is still active. " + "Make sure to stop all running application instances before running the reset tool."); } diff --git a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala index 8e10a875a420b..905d1133b3828 100644 --- a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala @@ -23,6 +23,7 @@ import java.util.Collections import java.util.Properties import org.easymock.EasyMock +import org.junit.Assert._ import org.junit.Before import org.junit.Test @@ -35,10 +36,11 @@ import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.TestUtils +import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.common.errors.GroupCoordinatorNotAvailableException +import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.common.serialization.StringDeserializer -import org.apache.kafka.clients.consumer.KafkaConsumer class DescribeConsumerGroupTest extends KafkaServerTestHarness { @@ -179,21 +181,8 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness { val opts = new ConsumerGroupCommandOptions(cgcArgs) val consumerGroupCommand = new KafkaConsumerGroupService(opts) - TestUtils.waitUntilTrue(() => { - try { - val (state, assignments) = consumerGroupCommand.describeGroup() - println(state == Some("Dead") && assignments == Some(List())) - state == Some("Dead") && assignments == Some(List()) - } catch { - case _: GroupCoordinatorNotAvailableException | _: IllegalArgumentException => - // Do nothing while the group initializes - false - case e: Throwable => - e.printStackTrace() - throw e - } - }, "Expected the state to be 'Dead' with no members in the group.") - + val (state, assignments) = consumerGroupCommand.describeGroup() + assertTrue("Expected the state to be 'Dead' with no members in the group.", state == Some("Dead") && assignments == Some(List())) consumerGroupCommand.close() } @@ -207,21 +196,13 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness { val consumerGroupCommand = new KafkaConsumerGroupService(opts) TestUtils.waitUntilTrue(() => { - try { - val (state, assignments) = consumerGroupCommand.describeGroup() - state == Some("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.clientId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.host.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) - } catch { - case _: GroupCoordinatorNotAvailableException | _: IllegalArgumentException => - // Do nothing while the group initializes - false - case e: Throwable => - throw e - } + val (state, assignments) = consumerGroupCommand.describeGroup() + state == Some("Stable") && + assignments.isDefined && + assignments.get.count(_.group == group) == 1 && + assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && + assignments.get.filter(_.group == group).head.clientId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && + assignments.get.filter(_.group == group).head.host.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) }, "Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe group results.") consumerGroupCommand.close() @@ -237,40 +218,24 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness { val consumerGroupCommand = new KafkaConsumerGroupService(opts) TestUtils.waitUntilTrue(() => { - try { - val (state, _) = consumerGroupCommand.describeGroup() - state == Some("Stable") - } catch { - case _: GroupCoordinatorNotAvailableException | _: IllegalArgumentException => - // Do nothing while the group initializes - false - case e: Throwable => - throw e - } + val (state, _) = consumerGroupCommand.describeGroup() + state == Some("Stable") }, "Expected the group to initially become stable.") // stop the consumer so the group has no active member anymore executor.shutdown() TestUtils.waitUntilTrue(() => { - try { - val (state, assignments) = consumerGroupCommand.describeGroup() - state == Some("Empty") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && // the member should be gone - assignments.get.filter(_.group == group).head.clientId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.host.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) - } catch { - case _: GroupCoordinatorNotAvailableException | _: IllegalArgumentException => - // Do nothing while the group initializes - false - case e: Throwable => - throw e - } finally { - consumerGroupCommand.close() - } + val (state, assignments) = consumerGroupCommand.describeGroup() + state == Some("Empty") && + assignments.isDefined && + assignments.get.count(_.group == group) == 1 && + assignments.get.filter(_.group == group).head.consumerId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && // the member should be gone + assignments.get.filter(_.group == group).head.clientId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && + assignments.get.filter(_.group == group).head.host.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) }, "Expected no active member in describe group results.") + + consumerGroupCommand.close() } @Test @@ -283,20 +248,12 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness { val consumerGroupCommand = new KafkaConsumerGroupService(opts) TestUtils.waitUntilTrue(() => { - try { - val (state, assignments) = consumerGroupCommand.describeGroup() - state == Some("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count{ x => x.group == group && x.partition.isDefined} == 1 && - assignments.get.count{ x => x.group == group && !x.partition.isDefined} == 1 - } catch { - case _: GroupCoordinatorNotAvailableException | _: IllegalArgumentException => - // Do nothing while the group initializes - false - case e: Throwable => - throw e - } + val (state, assignments) = consumerGroupCommand.describeGroup() + state == Some("Stable") && + assignments.isDefined && + assignments.get.count(_.group == group) == 2 && + assignments.get.count{ x => x.group == group && x.partition.isDefined} == 1 && + assignments.get.count{ x => x.group == group && !x.partition.isDefined} == 1 }, "Expected rows for consumers with no assigned partitions in describe group results.") consumerGroupCommand.close() @@ -315,24 +272,40 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness { val consumerGroupCommand = new KafkaConsumerGroupService(opts) TestUtils.waitUntilTrue(() => { - try { val (state, assignments) = consumerGroupCommand.describeGroup() state == Some("Stable") && assignments.isDefined && assignments.get.count(_.group == group) == 2 && assignments.get.count{ x => x.group == group && x.partition.isDefined} == 2 && assignments.get.count{ x => x.group == group && !x.partition.isDefined} == 0 - } catch { - case _: GroupCoordinatorNotAvailableException | _: IllegalArgumentException => - // Do nothing while the group initializes - false - case e: Throwable => - throw e - } }, "Expected two rows (one row per consumer) in describe group results.") consumerGroupCommand.close() } + + @Test + def testDescribeGroupWithNewConsumerWithShortInitializationTimeout() { + // run one consumer in the group consuming from a single-partition topic + val executor = new ConsumerGroupExecutor(brokerList, 1, group, topic) + + // set the group initialization timeout too low for the group to stabilize + val cgcArgs = Array("--bootstrap-server", brokerList, "--describe", "--group", "group", "--timeout", "10") + val opts = new ConsumerGroupCommandOptions(cgcArgs) + val consumerGroupCommand = new KafkaConsumerGroupService(opts) + + try { + val (state, assignments) = consumerGroupCommand.describeGroup() + fail("The consumer group command should fail due to low initialization timeout") + } catch { + case e: TimeoutException => + // OK + case e: Throwable => + fail("An unexpected exception occurred: " + e.getMessage) + throw e + } finally { + consumerGroupCommand.close() + } + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java index 3248b2ad95efb..4804bfbc771bd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java @@ -421,7 +421,7 @@ private void assertInternalTopicsGotDeleted(final String intermediateUserTopic) private class WaitUntilConsumerGroupGotClosed implements TestCondition { @Override public boolean conditionMet() { - return adminClient.describeConsumerGroup(APP_ID + testNo).consumers().get().isEmpty(); + return adminClient.describeConsumerGroup(APP_ID + testNo, 0).consumers().get().isEmpty(); } } From 2e92f9b2e4ab367f89a055a32a64915a38274065 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 3 Mar 2017 22:23:20 +0000 Subject: [PATCH 059/101] MINOR: Bump version to 0.11.0.0-SNAPSHOT There won't be a 0.10.3.0. Author: Ismael Juma Reviewers: Jason Gustafson Closes #2628 from ijuma/bump-version-to-0.11.0.0-SNAPSHOT --- gradle.properties | 2 +- kafka-merge-pr.py | 2 +- tests/kafkatest/__init__.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/gradle.properties b/gradle.properties index 7d5a8aaaa4d24..2211124fcf4b3 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,7 +16,7 @@ group=org.apache.kafka # NOTE: When you change this version number, you should also make sure to update # the version numbers in tests/kafkatest/__init__.py and kafka-merge-pr.py. -version=0.10.3.0-SNAPSHOT +version=0.11.0.0-SNAPSHOT scalaVersion=2.10.6 task=build org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index 240e78a3fc684..96a103e0d16c4 100755 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -72,7 +72,7 @@ DEV_BRANCH_NAME = "trunk" -DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "0.10.3.0") +DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "0.11.0.0") def get_json(url): try: diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index bc60fc8c980d1..8aff99acd7e7a 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 0.9.0.0-SNAPSHOT, this should be something like "0.9.0.0.dev0" -__version__ = '0.10.3.0.dev0' +__version__ = '0.11.0.0.dev0' From 4b1415c109fa75a01a11460aa80f0404baecac51 Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Fri, 3 Mar 2017 16:40:25 -0800 Subject: [PATCH 060/101] MINOR: Fix tests/docker/Dockerfile Fix tests/docker/Dockerfile to put the old Kafka distributions in the correct spot for tests. Also, run_tests.sh should exit with an error code if image rebuilding fails, rather than silently falling back to an older image. Author: Colin P. Mccabe Reviewers: Ewen Cheslack-Postava Closes #2613 from cmccabe/dockerfix --- tests/docker/Dockerfile | 8 ++++---- tests/docker/run_tests.sh | 8 +++++++- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 3a3df3242259a..78b2e32d35278 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -23,9 +23,9 @@ ADD ssh /root/.ssh RUN chmod 600 /root/.ssh/id_rsa RUN apt update && apt install -y unzip wget curl jq coreutils openssh-server net-tools vim openjdk-8-jdk python-pip python-dev libffi-dev libssl-dev RUN pip install -U pip && pip install --upgrade cffi ducktape==0.6.0 -RUN mkdir -p "/opt/kafka_2.10-0.8.2.2" && curl "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.8.2.2" -RUN mkdir -p "/opt/kafka_2.10-0.9.0.1" && curl "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.9.0.1" -RUN mkdir -p "/opt/kafka_2.10-0.10.0.1" && curl "${MIRROR}kafka/0.10.0.1/kafka_2.10-0.10.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.10.0.1" -RUN mkdir -p "/opt/kafka_2.10-0.10.1.1" && curl "${MIRROR}kafka/0.10.1.1/kafka_2.10-0.10.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka_2.10-0.10.1.1" +RUN mkdir -p "/opt/kafka-0.8.2.2" && curl -s "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.8.2.2" +RUN mkdir -p "/opt/kafka-0.9.0.1" && curl -s "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.9.0.1" +RUN mkdir -p "/opt/kafka-0.10.0.1" && curl -s "${MIRROR}kafka/0.10.0.1/kafka_2.10-0.10.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.0.1" +RUN mkdir -p "/opt/kafka-0.10.1.1" && curl -s "${MIRROR}kafka/0.10.1.1/kafka_2.10-0.10.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.1.1" CMD service ssh start && tail -f /dev/null diff --git a/tests/docker/run_tests.sh b/tests/docker/run_tests.sh index cc381cc33c2e4..11b551fad10c9 100755 --- a/tests/docker/run_tests.sh +++ b/tests/docker/run_tests.sh @@ -18,6 +18,11 @@ # TC_PATHS="tests/kafkatest/tests/streams tests/kafkatest/tests/tools" bash tests/docker/run_tests.sh set -x +die() { + echo $@ + exit 1 +} + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" TESTS_DIR=`dirname ${SCRIPT_DIR}` KAFKA_SRC=`dirname ${TESTS_DIR}` @@ -40,7 +45,8 @@ docker run --rm -it ${KAFKA_IMAGE} "true" if [[ $? != 0 || ${KAFKA_IMAGE_REBUILD} != "" ]]; then echo "kafka image ${KAFKA_IMAGE} does not exist. Building it from scratch." COMMIT_INFO=$(git describe HEAD) - docker build -t ${KAFKA_IMAGE} --label=commit_info=${COMMIT_INFO} ${SCRIPT_DIR} + docker build -t ${KAFKA_IMAGE} --label=commit_info=${COMMIT_INFO} ${SCRIPT_DIR} \ + || die "docker build failed" fi echo "Using kafka image: ${KAFKA_IMAGE}" From d2792e356ff9d3ed8b0484bf62ac98e85fdc26fa Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Sat, 4 Mar 2017 00:47:35 +0000 Subject: [PATCH 061/101] KAFKA-2857; MINOR: Follow up to MINOR: Fix ResetIntegrationTest test failures KAFKA-2857 follow-up. Author: Vahid Hashemian Reviewers: Jason Gustafson , Ismael Juma Closes #2636 from vahidhashemian/minor/kafka-2857-followup --- .../kafka/streams/integration/ResetIntegrationTest.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java index 4804bfbc771bd..88a85459a691d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java @@ -22,7 +22,7 @@ import kafka.utils.MockTime; import kafka.utils.ZkUtils; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.errors.GroupCoordinatorNotAvailableException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.security.JaasUtils; import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; @@ -122,9 +122,7 @@ public void cleanup() throws Exception { try { TestUtils.waitForCondition(consumerGroupInactive, TIMEOUT_MULTIPLIER * CLEANUP_CONSUMER_TIMEOUT, "Test consumer group active even after waiting " + (TIMEOUT_MULTIPLIER * CLEANUP_CONSUMER_TIMEOUT) + " ms."); - } catch (GroupCoordinatorNotAvailableException e) { - continue; - } catch (IllegalArgumentException e) { + } catch (TimeoutException e) { continue; } break; From d9b784e1470714c8b04e7c3d74f626a96ca1591e Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Sat, 4 Mar 2017 00:52:26 +0000 Subject: [PATCH 062/101] KAFKA-4796; Fix some findbugs warnings in Kafka Java client Author: Colin P. Mccabe Reviewers: Ismael Juma Closes #2593 from cmccabe/KAFKA-4796 --- checkstyle/suppressions.xml | 2 +- .../internals/DefaultPartitioner.java | 4 +- .../apache/kafka/common/config/ConfigDef.java | 7 +- .../kafka/common/protocol/Protocol.java | 5 -- .../kafka/common/protocol/types/Struct.java | 2 +- .../org/apache/kafka/common/utils/Bytes.java | 3 +- .../org/apache/kafka/common/utils/Utils.java | 67 +++++-------------- .../org/apache/kafka/clients/MockClient.java | 2 +- .../internals/AbstractCoordinatorTest.java | 2 +- .../internals/RecordAccumulatorTest.java | 2 +- .../producer/internals/SenderTest.java | 4 +- .../types/ProtocolSerializationTest.java | 12 ++++ .../common/security/JaasContextTest.java | 4 +- .../security/scram/ScramMessagesTest.java | 2 +- .../common/utils/AbstractIteratorTest.java | 2 +- .../apache/kafka/common/utils/UtilsTest.java | 64 ++++++++++++++++++ .../org/apache/kafka/test/TestSslUtils.java | 4 +- 17 files changed, 113 insertions(+), 75 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 9f133075c5285..a39695f934b6a 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -32,7 +32,7 @@ files=".*/protocol/Errors.java"/> + files="(Utils|KafkaLZ4BlockOutputStream).java"/> diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java index 086534a27ed5d..9d4ecbf151372 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java @@ -18,9 +18,9 @@ import java.util.List; import java.util.Map; -import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.clients.producer.Partitioner; @@ -73,7 +73,7 @@ public int partition(String topic, Object key, byte[] keyBytes, Object value, by private int nextValue(String topic) { AtomicInteger counter = topicCounterMap.get(topic); if (null == counter) { - counter = new AtomicInteger(new Random().nextInt()); + counter = new AtomicInteger(ThreadLocalRandom.current().nextInt()); AtomicInteger currentCounter = topicCounterMap.putIfAbsent(topic, counter); if (currentCounter != null) { counter = currentCounter; diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index f7cb8a9eeafdb..3396f6369b55b 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -72,7 +72,10 @@ * functionality for accessing configs. */ public class ConfigDef { - + /** + * A unique Java object which represents the lack of a default value.

    + * The 'new' here is intentional. + */ public static final Object NO_DEFAULT_VALUE = new String(""); private final Map configKeys; @@ -816,7 +819,7 @@ public static Range between(Number min, Number max) { public void ensureValid(String name, Object o) { if (o == null) - throw new ConfigException(name, o, "Value must be non-null"); + throw new ConfigException(name, null, "Value must be non-null"); Number n = (Number) o; if (min != null && n.doubleValue() < min.doubleValue()) throw new ConfigException(name, o, "Value must be at least " + min); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 25d380b6df4bb..3343133b7e44e 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -1090,11 +1090,6 @@ private static void schemaToBnfHtml(Schema schema, StringBuilder b, int indentSi Type innerType = ((ArrayOf) field.type).type(); if (!subTypes.containsKey(field.name)) subTypes.put(field.name, innerType); - } else if (field.type instanceof Schema) { - b.append(field.name); - b.append(" "); - if (!subTypes.containsKey(field.name)) - subTypes.put(field.name, field.type); } else { b.append(field.name); b.append(" "); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index c32aea7c0dcb4..325690d56daeb 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -349,7 +349,7 @@ public boolean equals(Object obj) { } else { Object thisField = this.get(f); Object otherField = other.get(f); - result = (thisField == null && otherField == null) || thisField.equals(otherField); + return (thisField == null) ? (otherField == null) : thisField.equals(otherField); } if (!result) return false; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java index e28d925acbc06..4099155b35b75 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.utils; +import java.io.Serializable; import java.util.Arrays; import java.util.Comparator; @@ -138,7 +139,7 @@ private static String toString(final byte[] b, int off, int len) { */ public final static Comparator BYTES_LEXICO_COMPARATOR = new LexicographicByteArrayComparator(); - private interface ByteArrayComparator extends Comparator { + private interface ByteArrayComparator extends Comparator, Serializable { int compare(final byte[] buffer1, int offset1, int length1, final byte[] buffer2, int offset2, int length2); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 20ab814e49463..ed5eddb192536 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -102,16 +102,6 @@ public static byte[] utf8(String string) { } } - /** - * Read an unsigned integer from the current position in the buffer, incrementing the position by 4 bytes - * - * @param buffer The buffer to read from - * @return The integer read, as a long to avoid signedness - */ - public static long readUnsignedInt(ByteBuffer buffer) { - return buffer.getInt() & 0xffffffffL; - } - /** * Read an unsigned integer from the given position without modifying the buffers position * @@ -130,27 +120,12 @@ public static long readUnsignedInt(ByteBuffer buffer, int index) { * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) */ public static int readUnsignedIntLE(InputStream in) throws IOException { - return (in.read() << 8 * 0) - | (in.read() << 8 * 1) - | (in.read() << 8 * 2) - | (in.read() << 8 * 3); + return in.read() + | (in.read() << 8) + | (in.read() << 16) + | (in.read() << 24); } - /** - * Get the little-endian value of an integer as a byte array. - * @param val The value to convert to a little-endian array - * @return The little-endian encoded array of bytes for the value - */ - public static byte[] toArrayLE(int val) { - return new byte[] { - (byte) (val >> 8 * 0), - (byte) (val >> 8 * 1), - (byte) (val >> 8 * 2), - (byte) (val >> 8 * 3) - }; - } - - /** * Read an unsigned integer stored in little-endian format from a byte array * at a given offset. @@ -160,20 +135,10 @@ public static byte[] toArrayLE(int val) { * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) */ public static int readUnsignedIntLE(byte[] buffer, int offset) { - return (buffer[offset++] << 8 * 0) - | (buffer[offset++] << 8 * 1) - | (buffer[offset++] << 8 * 2) - | (buffer[offset] << 8 * 3); - } - - /** - * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. - * - * @param buffer The buffer to write to - * @param value The value to write - */ - public static void writeUnsignedInt(ByteBuffer buffer, long value) { - buffer.putInt((int) (value & 0xffffffffL)); + return (buffer[offset] << 0 & 0xff) + | ((buffer[offset + 1] & 0xff) << 8) + | ((buffer[offset + 2] & 0xff) << 16) + | ((buffer[offset + 3] & 0xff) << 24); } /** @@ -194,10 +159,10 @@ public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) { * @param value The value to write */ public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException { - out.write(value >>> 8 * 0); - out.write(value >>> 8 * 1); - out.write(value >>> 8 * 2); - out.write(value >>> 8 * 3); + out.write(value); + out.write(value >>> 8); + out.write(value >>> 16); + out.write(value >>> 24); } /** @@ -209,10 +174,10 @@ public static void writeUnsignedIntLE(OutputStream out, int value) throws IOExce * @param value The value to write */ public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { - buffer[offset++] = (byte) (value >>> 8 * 0); - buffer[offset++] = (byte) (value >>> 8 * 1); - buffer[offset++] = (byte) (value >>> 8 * 2); - buffer[offset] = (byte) (value >>> 8 * 3); + buffer[offset] = (byte) value; + buffer[offset + 1] = (byte) (value >>> 8); + buffer[offset + 2] = (byte) (value >>> 16); + buffer[offset + 3] = (byte) (value >>> 24); } diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index f97e407ed8d44..7e05881243350 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -47,7 +47,7 @@ public boolean matches(AbstractRequest body) { } }; - private class FutureResponse { + private static class FutureResponse { public final AbstractResponse responseBody; public final boolean disconnected; public final RequestMatcher requestMatcher; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 3eb6561b39a0c..45ee29a53b942 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -483,7 +483,7 @@ private SyncGroupResponse syncGroupResponse(Errors error) { return new SyncGroupResponse(error, ByteBuffer.allocate(0)); } - public class DummyCoordinator extends AbstractCoordinator { + public static class DummyCoordinator extends AbstractCoordinator { private int onJoinPrepareInvokes = 0; private int onJoinCompleteInvokes = 0; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index b96594f2b8cf3..1cb510e3f2555 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -84,7 +84,7 @@ public void teardown() { public void testFull() throws Exception { long now = time.milliseconds(); int batchSize = 1024; - RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * batchSize, CompressionType.NONE, 10L, 100L, metrics, time); + RecordAccumulator accum = new RecordAccumulator(batchSize, 10L * batchSize, CompressionType.NONE, 10L, 100L, metrics, time); int appends = batchSize / msgSize; for (int i = 0; i < appends; i++) { // append to the first batch diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index e9a7188739b33..50ea219015c36 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -151,7 +151,7 @@ public void testRetries() throws Exception { sender.run(time.milliseconds()); // connect sender.run(time.milliseconds()); // send produce request String id = client.requests().peek().destination(); - Node node = new Node(Integer.valueOf(id), "localhost", 0); + Node node = new Node(Integer.parseInt(id), "localhost", 0); assertEquals(1, client.inFlightRequestCount()); assertTrue("Client ready status should be true", client.isReady(node, 0L)); client.disconnect(id); @@ -210,7 +210,7 @@ public void testSendInOrder() throws Exception { sender.run(time.milliseconds()); // send produce request String id = client.requests().peek().destination(); assertEquals(ApiKeys.PRODUCE, client.requests().peek().requestBuilder().apiKey()); - Node node = new Node(Integer.valueOf(id), "localhost", 0); + Node node = new Node(Integer.parseInt(id), "localhost", 0); assertEquals(1, client.inFlightRequestCount()); assertTrue("Client ready status should be true", client.isReady(node, 0L)); diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index 74c930290cef3..1c14e829a3005 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.protocol.types; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; @@ -258,4 +259,15 @@ private void check(Type type, Object obj) { assertEquals("The object read back should be the same as what was written.", obj, result); } + @Test + public void testStructEquals() { + Schema schema = new Schema(new Field("field1", Type.NULLABLE_STRING), new Field("field2", Type.NULLABLE_STRING)); + Struct emptyStruct1 = new Struct(schema); + Struct emptyStruct2 = new Struct(schema); + assertEquals(emptyStruct1, emptyStruct2); + + Struct mostlyEmptyStruct = new Struct(schema).set("field1", "foo"); + assertNotEquals(emptyStruct1, mostlyEmptyStruct); + assertNotEquals(mostlyEmptyStruct, emptyStruct1); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java b/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java index 8d98a113c3372..30799c5f91591 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/JaasContextTest.java @@ -58,8 +58,8 @@ public void setUp() throws IOException { } @After - public void tearDown() { - jaasConfigFile.delete(); + public void tearDown() throws Exception { + Files.delete(jaasConfigFile.toPath()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java index 53939ef4c7d4a..de97ce2e144db 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/scram/ScramMessagesTest.java @@ -252,7 +252,7 @@ public void validServerFinalMessage() throws SaslException { checkServerFinalMessage(m, null, serverSignature); // Default format used by Kafka clients for final message with error - str = String.format("e=other-error", serverSignature); + str = "e=other-error"; m = createScramMessage(ServerFinalMessage.class, str); checkServerFinalMessage(m, "other-error", null); m = new ServerFinalMessage(m.toBytes()); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java index 96adbbea31c67..5ddab740c5e6a 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java @@ -52,7 +52,7 @@ public void testEmptyIterator() { iter.next(); } - class ListIterator extends AbstractIterator { + static class ListIterator extends AbstractIterator { private List list; private int position = 0; diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index c3f69fa4b507c..5f36c1cfcd973 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.utils; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.EOFException; import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; @@ -35,6 +37,7 @@ import static org.apache.kafka.common.utils.Utils.formatAddress; import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getPort; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -333,4 +336,65 @@ static void checkException(IOException e, TestCloseable... closeablesWithExcepti assertEquals(closeablesWithException[i].closeException, suppressed[i - 1]); } } + + @Test + public void testReadUnsignedIntLEFromArray() { + byte[] array1 = {0x01, 0x02, 0x03, 0x04, 0x05}; + assertEquals(0x04030201, Utils.readUnsignedIntLE(array1, 0)); + assertEquals(0x05040302, Utils.readUnsignedIntLE(array1, 1)); + + byte[] array2 = {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, (byte) 0xf5, (byte) 0xf6}; + assertEquals(0xf4f3f2f1, Utils.readUnsignedIntLE(array2, 0)); + assertEquals(0xf6f5f4f3, Utils.readUnsignedIntLE(array2, 2)); + } + + @Test + public void testReadUnsignedIntLEFromInputStream() throws IOException { + byte[] array1 = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09}; + ByteArrayInputStream is1 = new ByteArrayInputStream(array1); + assertEquals(0x04030201, Utils.readUnsignedIntLE(is1)); + assertEquals(0x08070605, Utils.readUnsignedIntLE(is1)); + + byte[] array2 = {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, (byte) 0xf5, (byte) 0xf6, (byte) 0xf7, (byte) 0xf8}; + ByteArrayInputStream is2 = new ByteArrayInputStream(array2); + assertEquals(0xf4f3f2f1, Utils.readUnsignedIntLE(is2)); + assertEquals(0xf8f7f6f5, Utils.readUnsignedIntLE(is2)); + } + + @Test + public void testWriteUnsignedIntLEToArray() { + int value1 = 0x04030201; + + byte[] array1 = new byte[4]; + Utils.writeUnsignedIntLE(array1, 0, value1); + assertArrayEquals(new byte[] {0x01, 0x02, 0x03, 0x04}, array1); + + array1 = new byte[8]; + Utils.writeUnsignedIntLE(array1, 2, value1); + assertArrayEquals(new byte[] {0, 0, 0x01, 0x02, 0x03, 0x04, 0, 0}, array1); + + int value2 = 0xf4f3f2f1; + + byte[] array2 = new byte[4]; + Utils.writeUnsignedIntLE(array2, 0, value2); + assertArrayEquals(new byte[] {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4}, array2); + + array2 = new byte[8]; + Utils.writeUnsignedIntLE(array2, 2, value2); + assertArrayEquals(new byte[] {0, 0, (byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, 0, 0}, array2); + } + + @Test + public void testWriteUnsignedIntLEToOutputStream() throws IOException { + int value1 = 0x04030201; + ByteArrayOutputStream os1 = new ByteArrayOutputStream(); + Utils.writeUnsignedIntLE(os1, value1); + Utils.writeUnsignedIntLE(os1, value1); + assertArrayEquals(new byte[] {0x01, 0x02, 0x03, 0x04, 0x01, 0x02, 0x03, 0x04}, os1.toByteArray()); + + int value2 = 0xf4f3f2f1; + ByteArrayOutputStream os2 = new ByteArrayOutputStream(); + Utils.writeUnsignedIntLE(os2, value2); + assertArrayEquals(new byte[] {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4}, os2.toByteArray()); + } } diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index 7b78d3e500057..f4f88184ec3a7 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -152,10 +152,8 @@ public static void createKeyStore(String filename, public static void createTrustStore( String filename, Password password, Map certs) throws GeneralSecurityException, IOException { KeyStore ks = KeyStore.getInstance("JKS"); - try { - FileInputStream in = new FileInputStream(filename); + try (FileInputStream in = new FileInputStream(filename)) { ks.load(in, password.value().toCharArray()); - in.close(); } catch (EOFException e) { ks = createEmptyKeyStore(); } From 5781feb527766fbb3620cb601ed453dd8086e0c2 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 4 Mar 2017 15:22:06 +0000 Subject: [PATCH 063/101] KAFKA-3182; Fix testSocketsCloseOnShutdown transient failures * Turned off Nagle on the sending sockets to force the socket to physically acknowledge after the first write in `sendRequest` * Added a `200ms` delay between write attempts (necessary on Linux, but not Mac) Author: Armin Braun Reviewers: Ismael Juma Closes #2632 from original-brownbear/KAFKA-3182 --- .../src/test/scala/unit/kafka/network/SocketServerTest.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 387560465af2b..17056c917928c 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -181,13 +181,16 @@ class SocketServerTest extends JUnitSuite { def testSocketsCloseOnShutdown() { // open a connection val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + plainSocket.setTcpNoDelay(true) val traceSocket = connect(protocol = SecurityProtocol.TRACE) + traceSocket.setTcpNoDelay(true) val bytes = new Array[Byte](40) // send a request first to make sure the connection has been picked up by the socket server sendRequest(plainSocket, bytes, Some(0)) sendRequest(traceSocket, bytes, Some(0)) processRequest(server.requestChannel) - + // the following sleep is necessary to reliably detect the connection close when we send data below + Thread.sleep(200L) // make sure the sockets are open server.acceptors.values.map(acceptor => assertFalse(acceptor.serverChannel.socket.isClosed)) // then shutdown the server From adb70da13e18eb652e734887b430ac0ecbc5f9e6 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Sat, 4 Mar 2017 11:38:57 -0800 Subject: [PATCH 064/101] KAFKA-4820; ConsumerNetworkClient.send() should not require global lock Author: Dong Lin Reviewers: Jason Gustafson , Ismael Juma , Jiangjie Qin Closes #2619 from lindong28/KAFKA-4820 --- .../kafka/clients/consumer/KafkaConsumer.java | 2 +- .../internals/ConsumerNetworkClient.java | 210 +++++++++++++----- .../clients/consumer/internals/Fetcher.java | 2 +- 3 files changed, 153 insertions(+), 61 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index b4514c55d370a..51b00afd42d05 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1004,7 +1004,7 @@ public ConsumerRecords poll(long timeout) { // // NOTE: since the consumed position has already been updated, we must not allow // wakeups or any other errors to be triggered prior to returning the fetched records. - if (fetcher.sendFetches() > 0 || client.pendingRequestCount() > 0) + if (fetcher.sendFetches() > 0 || client.hasPendingRequest()) client.pollNoWakeup(); if (this.interceptors == null) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 8781676cba467..2fa766782e2b7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -35,10 +35,13 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.kafka.common.errors.InterruptException; @@ -55,7 +58,7 @@ public class ConsumerNetworkClient implements Closeable { // the mutable state of this class is protected by the object's monitor (excluding the wakeup // flag and the request completion queue below). private final KafkaClient client; - private final Map> unsent = new HashMap<>(); + private final UnsentRequests unsent = new UnsentRequests(); private final Metadata metadata; private final Time time; private final long retryBackoffMs; @@ -99,24 +102,13 @@ public RequestFuture send(Node node, AbstractRequest.Builder RequestFutureCompletionHandler completionHandler = new RequestFutureCompletionHandler(); ClientRequest clientRequest = client.newClientRequest(node.idString(), requestBuilder, now, true, completionHandler); - put(node, clientRequest); + unsent.put(node, clientRequest); // wakeup the client in case it is blocking in poll so that we can send the queued request client.wakeup(); return completionHandler.future; } - private void put(Node node, ClientRequest request) { - synchronized (this) { - List nodeUnsent = unsent.get(node); - if (nodeUnsent == null) { - nodeUnsent = new ArrayList<>(); - unsent.put(node, nodeUnsent); - } - nodeUnsent.add(request); - } - } - public Node leastLoadedNode() { synchronized (this) { return client.leastLoadedNode(time.milliseconds()); @@ -280,12 +272,12 @@ public boolean awaitPendingRequests(Node node, long timeoutMs) { long startMs = time.milliseconds(); long remainingMs = timeoutMs; - while (pendingRequestCount(node) > 0 && remainingMs > 0) { + while (hasPendingRequest(node) && remainingMs > 0) { poll(remainingMs); remainingMs = timeoutMs - (time.milliseconds() - startMs); } - return pendingRequestCount(node) == 0; + return !hasPendingRequest(node); } /** @@ -296,9 +288,21 @@ public boolean awaitPendingRequests(Node node, long timeoutMs) { */ public int pendingRequestCount(Node node) { synchronized (this) { - List pending = unsent.get(node); - int unsentCount = pending == null ? 0 : pending.size(); - return unsentCount + client.inFlightRequestCount(node.idString()); + return unsent.getRequestCount(node) + client.inFlightRequestCount(node.idString()); + } + } + + /** + * Check whether there is pending request to the given node. This includes both request that + * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission. + * @param node The node in question + * @return A boolean indicating whether there is pending request + */ + public boolean hasPendingRequest(Node node) { + if (unsent.hasRequest(node)) + return true; + synchronized (this) { + return client.inFlightRequestCount(node.idString()) > 0; } } @@ -309,10 +313,20 @@ public int pendingRequestCount(Node node) { */ public int pendingRequestCount() { synchronized (this) { - int total = 0; - for (List requests: unsent.values()) - total += requests.size(); - return total + client.inFlightRequestCount(); + return unsent.getRequestCount() + client.inFlightRequestCount(); + } + } + + /** + * Check whether there is pending request. This includes both requests that + * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission. + * @return A boolean indicating whether there is pending request + */ + public boolean hasPendingRequest() { + if (unsent.hasRequest()) + return true; + synchronized (this) { + return client.inFlightRequestCount() > 0; } } @@ -337,19 +351,17 @@ private void checkDisconnects(long now) { // by NetworkClient, so we just need to check whether connections for any of the unsent // requests have been disconnected; if they have, then we complete the corresponding future // and set the disconnect flag in the ClientResponse - Iterator>> iterator = unsent.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry> requestEntry = iterator.next(); - Node node = requestEntry.getKey(); + Set nodes = unsent.getNodes(); + for (Node node: nodes) { if (client.connectionFailed(node)) { // Remove entry before invoking request callback to avoid callbacks handling // coordinator failures traversing the unsent list again. - iterator.remove(); - for (ClientRequest request : requestEntry.getValue()) { + List requests = unsent.remove(node); + for (ClientRequest request : requests) { RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); handler.onComplete(new ClientResponse(request.makeHeader(request.requestBuilder().desiredOrLatestVersion()), - request.callback(), request.destination(), request.createdTimeMs(), now, true, - null, null)); + request.callback(), request.destination(), request.createdTimeMs(), now, true, + null, null)); } } } @@ -357,21 +369,10 @@ private void checkDisconnects(long now) { private void failExpiredRequests(long now) { // clear all expired unsent requests and fail their corresponding futures - Iterator>> iterator = unsent.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry> requestEntry = iterator.next(); - Iterator requestIterator = requestEntry.getValue().iterator(); - while (requestIterator.hasNext()) { - ClientRequest request = requestIterator.next(); - if (request.createdTimeMs() < now - unsentExpiryMs) { - RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); - handler.onFailure(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); - requestIterator.remove(); - } else - break; - } - if (requestEntry.getValue().isEmpty()) - iterator.remove(); + List expiredRequests = unsent.removeExpiredRequests(now, unsentExpiryMs); + for (ClientRequest request: expiredRequests) { + RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); + handler.onFailure(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); } } @@ -379,11 +380,9 @@ public void failUnsentRequests(Node node, RuntimeException e) { // clear unsent requests to node and fail their corresponding futures synchronized (this) { List unsentRequests = unsent.remove(node); - if (unsentRequests != null) { - for (ClientRequest unsentRequest : unsentRequests) { - RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) unsentRequest.callback(); - handler.onFailure(e); - } + for (ClientRequest unsentRequest : unsentRequests) { + RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) unsentRequest.callback(); + handler.onFailure(e); } } @@ -394,15 +393,25 @@ public void failUnsentRequests(Node node, RuntimeException e) { private boolean trySend(long now) { // send any requests that can be sent now boolean requestsSent = false; - for (Map.Entry> requestEntry: unsent.entrySet()) { - Node node = requestEntry.getKey(); - Iterator iterator = requestEntry.getValue().iterator(); - while (iterator.hasNext()) { - ClientRequest request = iterator.next(); - if (client.ready(node, now)) { - client.send(request, now); - iterator.remove(); - requestsSent = true; + Set nodes = unsent.getNodes(); + for (Node node: nodes) { + if (client.ready(node, now)) { + // Remove entry before invoking request callback to avoid callbacks handling + // coordinator failures traversing the unsent list again. + List requests = unsent.remove(node); + try { + Iterator iterator = requests.iterator(); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + if (!client.ready(node, now)) + break; + client.send(request, now); + requestsSent = true; + iterator.remove(); + } + } finally { + if (!requests.isEmpty()) + unsent.put(node, requests); } } } @@ -527,4 +536,87 @@ public interface PollCondition { boolean shouldBlock(); } + + /* + * A threadsafe helper class to hold requests per node that has not been sent yet + */ + private final static class UnsentRequests { + private final Map> unsent; + + public UnsentRequests() { + unsent = new HashMap<>(); + } + + public synchronized void put(Node node, List requests) { + List nodeUnsent = unsent.get(node); + if (nodeUnsent == null) { + nodeUnsent = new ArrayList<>(); + unsent.put(node, nodeUnsent); + } + nodeUnsent.addAll(requests); + } + + public synchronized void put(Node node, ClientRequest request) { + List nodeUnsent = unsent.get(node); + if (nodeUnsent == null) { + nodeUnsent = new ArrayList<>(); + unsent.put(node, nodeUnsent); + } + nodeUnsent.add(request); + } + + public synchronized int getRequestCount(Node node) { + List requests = unsent.get(node); + return requests == null ? 0 : requests.size(); + } + + public synchronized int getRequestCount() { + int total = 0; + for (List requests : unsent.values()) + total += requests.size(); + return total; + } + + public synchronized boolean hasRequest(Node node) { + List requests = unsent.get(node); + return requests != null && !requests.isEmpty(); + } + + public synchronized boolean hasRequest() { + for (List requests : unsent.values()) + if (!requests.isEmpty()) + return true; + return false; + } + + public synchronized List removeExpiredRequests(long now, long unsentExpiryMs) { + List expiredRequests = new ArrayList<>(); + Iterator>> iterator = unsent.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> requestEntry = iterator.next(); + Iterator requestIterator = requestEntry.getValue().iterator(); + while (requestIterator.hasNext()) { + ClientRequest request = requestIterator.next(); + if (request.createdTimeMs() < now - unsentExpiryMs) { + expiredRequests.add(request); + requestIterator.remove(); + } else + break; + } + if (requestEntry.getValue().isEmpty()) + iterator.remove(); + } + return expiredRequests; + } + + public synchronized List remove(Node node) { + List requests = unsent.remove(node); + return requests == null ? Collections.emptyList() : requests; + } + + public synchronized Set getNodes() { + return new HashSet<>(unsent.keySet()); + } + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 8a8952cb125b6..536e4e823768b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -713,7 +713,7 @@ private Map createFetchRequests() { Node node = cluster.leaderFor(partition); if (node == null) { metadata.requestUpdate(); - } else if (this.client.pendingRequestCount(node) == 0) { + } else if (!this.client.hasPendingRequest(node)) { // if there is a leader and no in-flight requests, issue a new fetch LinkedHashMap fetch = fetchable.get(node); if (fetch == null) { From b7378d567fffd06395f5babc36cebd64bdf539d1 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Sat, 4 Mar 2017 20:55:16 -0800 Subject: [PATCH 065/101] MINOR: Standardised benchmark params for consumer and streams There were some minor differences in the basic consumer config and streams config that are now rectified. In addition, in AWS environments the socket size makes a big difference to performance and I've tuned it up accordingly. I've also increased the number of records now that perf is higher. Author: Eno Thereska Reviewers: Guozhang Wang Closes #2634 from enothereska/minor-standardize-params --- .../kafka/streams/perf/SimpleBenchmark.java | 17 ++++++++++++++++- .../streams/streams_simple_benchmark_test.py | 2 +- tests/kafkatest/services/streams.py | 2 +- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index cf593e2104c6c..7a36d7053b707 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -104,6 +104,9 @@ public byte[] apply(final byte[] value1, final byte[] value2) { private static int processedRecords = 0; private static long processedBytes = 0; private static final int VALUE_SIZE = 100; + private static final long POLL_MS = 500L; + private static final int MAX_POLL_RECORDS = 1000; + private static final int SOCKET_SIZE_BYTES = 1 * 1024 * 1024; private static final Serde BYTE_SERDE = Serdes.ByteArray(); private static final Serde INTEGER_SERDE = Serdes.Integer(); @@ -207,8 +210,13 @@ private Properties setStreamProperties(final String applicationId) { props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + // the socket buffer needs to be large, especially when running in AWS with + // high latency. if running locally the default is fine. + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, SOCKET_SIZE_BYTES); props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.ByteArray().getClass()); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, MAX_POLL_RECORDS); + props.put(StreamsConfig.POLL_MS_CONFIG, POLL_MS); return props; } @@ -218,9 +226,16 @@ private Properties setProduceConsumeProperties(final String clientId) { props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + // the socket buffer needs to be large, especially when running in AWS with + // high latency. if running locally the default is fine. + props.put(ProducerConfig.SEND_BUFFER_CONFIG, SOCKET_SIZE_BYTES); props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + // the socket buffer needs to be large, especially when running in AWS with + // high latency. if running locally the default is fine. + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, SOCKET_SIZE_BYTES); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, MAX_POLL_RECORDS); return props; } @@ -516,7 +531,7 @@ public void consume(String topic) throws Exception { long startTime = System.currentTimeMillis(); while (true) { - ConsumerRecords records = consumer.poll(500); + ConsumerRecords records = consumer.poll(POLL_MS); if (records.isEmpty()) { if (processedRecords == numRecords) break; diff --git a/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py b/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py index c1db8c813ee0b..c9f970e1fd81f 100644 --- a/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py +++ b/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py @@ -30,7 +30,7 @@ class StreamsSimpleBenchmarkTest(Test): def __init__(self, test_context): super(StreamsSimpleBenchmarkTest, self).__init__(test_context) - self.num_records = 2000000L + self.num_records = 10000000L self.replication = 1 diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 1e1c676151879..4f8f1a3f0733e 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -97,7 +97,7 @@ def abortThenRestart(self): self.logger.info("Restarting Kafka Streams on " + str(node.account)) self.start_node(node) - def wait(self, timeout_sec=360): + def wait(self, timeout_sec=720): for node in self.nodes: self.wait_node(node, timeout_sec) From 580bebe09798b8f0062a2a1566a1d5836ecad4bb Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Sun, 5 Mar 2017 10:01:30 -0800 Subject: [PATCH 066/101] KAFKA-4826: Fix some findbugs warnings in Kafka Streams Author: Colin P. Mccabe Reviewers: Matthias J. Sax, Guozhang Wang Closes #2623 from cmccabe/KAFKA-4826 --- .../org/apache/kafka/streams/kstream/Window.java | 4 +++- .../kstream/internals/KStreamWindowAggregate.java | 6 +++--- .../processor/internals/InternalTopicManager.java | 10 ++++++---- .../processor/internals/StoreChangelogReader.java | 6 ++++-- .../processor/internals/StreamsKafkaClient.java | 6 ++++-- .../processor/internals/assignment/ClientState.java | 9 +++++---- .../streams/state/internals/OffsetCheckpoint.java | 3 ++- .../kafka/streams/state/internals/RocksDBStore.java | 13 ++++++++++--- .../apache/kafka/streams/kstream/WindowTest.java | 5 +++++ 9 files changed, 42 insertions(+), 20 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index 2365f0e441159..c342112f1b0ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -86,7 +86,9 @@ public boolean equals(final Object obj) { if (obj == this) { return true; } - + if (obj == null) { + return false; + } if (getClass() != obj.getClass()) { return false; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index e2ba512bce81d..9730511cd3f2e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -116,11 +116,11 @@ public void process(K key, V value) { } // create the new window for the rest of unmatched window that do not exist yet - for (long windowStartMs : matchedWindows.keySet()) { + for (Map.Entry entry : matchedWindows.entrySet()) { T oldAgg = initializer.apply(); T newAgg = aggregator.apply(key, value, oldAgg); - windowStore.put(key, newAgg, windowStartMs); - tupleForwarder.maybeForward(new Windowed<>(key, matchedWindows.get(windowStartMs)), newAgg, oldAgg); + windowStore.put(key, newAgg, entry.getKey()); + tupleForwarder.maybeForward(new Windowed<>(key, entry.getValue()), newAgg, oldAgg); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 821881970befb..d8575e92ab348 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -94,15 +94,17 @@ public void close() { private Map validateTopicPartitions(final Map topicsPartitionsMap, final Map existingTopicNamesPartitions) { final Map topicsToBeCreated = new HashMap<>(); - for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { + for (Map.Entry entry : topicsPartitionsMap.entrySet()) { + InternalTopicConfig topic = entry.getKey(); + Integer partition = entry.getValue(); if (existingTopicNamesPartitions.containsKey(topic.name())) { - if (!existingTopicNamesPartitions.get(topic.name()).equals(topicsPartitionsMap.get(topic))) { + if (!existingTopicNamesPartitions.get(topic.name()).equals(partition)) { throw new StreamsException("Existing internal topic " + topic.name() + " has invalid partitions." + - " Expected: " + topicsPartitionsMap.get(topic) + " Actual: " + existingTopicNamesPartitions.get(topic.name()) + + " Expected: " + partition + " Actual: " + existingTopicNamesPartitions.get(topic.name()) + ". Use 'kafka.tools.StreamsResetter' tool to clean up invalid topics before processing."); } } else { - topicsToBeCreated.put(topic, topicsPartitionsMap.get(topic)); + topicsToBeCreated.put(topic, partition); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 85104fa0133b1..f95ea4a404e0f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -102,9 +102,11 @@ public void restore() { // remove any partitions where we already have all of the data final Map needsRestoring = new HashMap<>(); - for (final TopicPartition topicPartition : endOffsets.keySet()) { + for (final Map.Entry entry : endOffsets.entrySet()) { + TopicPartition topicPartition = entry.getKey(); + Long offset = entry.getValue(); final StateRestorer restorer = stateRestorers.get(topicPartition); - if (restorer.checkpoint() >= endOffsets.get(topicPartition)) { + if (restorer.checkpoint() >= offset) { restorer.setRestoredOffset(restorer.checkpoint()); } else { needsRestoring.put(topicPartition, restorer); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 4ca0b4d51ae79..c493d7488377b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -144,14 +144,16 @@ public void createTopics(final Map topicsMap, fina final long windowChangeLogAdditionalRetention, final MetadataResponse metadata) { final Map topicRequestDetails = new HashMap<>(); - for (InternalTopicConfig internalTopicConfig:topicsMap.keySet()) { + for (Map.Entry entry : topicsMap.entrySet()) { + InternalTopicConfig internalTopicConfig = entry.getKey(); + Integer partitions = entry.getValue(); final Properties topicProperties = internalTopicConfig.toProperties(windowChangeLogAdditionalRetention); final Map topicConfig = new HashMap<>(); for (String key : topicProperties.stringPropertyNames()) { topicConfig.put(key, topicProperties.getProperty(key)); } final CreateTopicsRequest.TopicDetails topicDetails = new CreateTopicsRequest.TopicDetails( - topicsMap.get(internalTopicConfig), + partitions, (short) replicationFactor, topicConfig); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java index d5f8ccf1280e6..99bd29ee7956b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java @@ -117,11 +117,12 @@ boolean hasMoreAvailableCapacityThan(final ClientState other) { final double otherLoad = (double) other.assignedTaskCount() / other.capacity; final double thisLoad = (double) assignedTaskCount() / capacity; - if (thisLoad == otherLoad) { + if (thisLoad < otherLoad) + return true; + else if (thisLoad > otherLoad) + return false; + else return capacity > other.capacity; - } - - return thisLoad < otherLoad; } Set previousStandbyTasks() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java index 57c7a85ce7cf6..b676421e4e4b5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java @@ -28,6 +28,7 @@ import java.io.FileReader; import java.io.IOException; import java.io.OutputStreamWriter; +import java.nio.file.Files; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -168,7 +169,7 @@ private int readInt(BufferedReader reader) throws IOException { * @throws IOException if there is any IO exception during delete */ public void delete() throws IOException { - file.delete(); + Files.delete(file.toPath()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index db4a03f33fa6e..823ad47ee5463 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; @@ -43,6 +44,8 @@ import org.rocksdb.WriteOptions; import java.io.File; +import java.io.IOException; +import java.nio.file.Files; import java.util.Comparator; import java.util.HashSet; import java.util.List; @@ -141,7 +144,11 @@ public void openDB(ProcessorContext context) { valueSerde == null ? (Serde) context.valueSerde() : valueSerde); this.dbDir = new File(new File(context.stateDir(), parentDir), this.name); - this.db = openDB(this.dbDir, this.options, TTL_SECONDS); + try { + this.db = openDB(this.dbDir, this.options, TTL_SECONDS); + } catch (IOException e) { + throw new StreamsException(e); + } } public void init(ProcessorContext context, StateStore root) { @@ -160,10 +167,10 @@ public void restore(byte[] key, byte[] value) { open = true; } - private RocksDB openDB(File dir, Options options, int ttl) { + private RocksDB openDB(File dir, Options options, int ttl) throws IOException { try { if (ttl == TTL_NOT_USED) { - dir.getParentFile().mkdirs(); + Files.createDirectories(dir.getParentFile().toPath()); return RocksDB.open(options, dir.getAbsolutePath()); } else { throw new UnsupportedOperationException("Change log is not supported for store " + this.name + " since it is TTL based."); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java index e7a579e5621ba..4b3c84da905de 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java @@ -66,6 +66,11 @@ public void shouldBeEqualIfStartAndEndSame() { assertEquals(window2, window); } + @Test + public void shouldNotBeEqualIfNull() { + assertNotEquals(window, null); + } + @Test public void shouldNotBeEqualIfStartOrEndIsDifferent() { assertNotEquals(window, new TestWindow(0, window.endMs)); From 3e69ef6e3c41a83bdec1e2a37073a6af2f44e2a2 Mon Sep 17 00:00:00 2001 From: Akhilesh Naidu Date: Sun, 5 Mar 2017 10:21:04 -0800 Subject: [PATCH 067/101] KAFKA-4276: Add REST configuration in connector properties Addition of REST configuration in connect-distributed.properties config file gwenshap ewencp - Please review. Author: Akhilesh Naidu Reviewers: Gwen Shapira Closes #2505 from akhilesh1194/JIRA_KAFKA-4276 --- config/connect-distributed.properties | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/config/connect-distributed.properties b/config/connect-distributed.properties index 931b85352889e..b0092bb9b5446 100644 --- a/config/connect-distributed.properties +++ b/config/connect-distributed.properties @@ -48,4 +48,13 @@ config.storage.topic=connect-configs status.storage.topic=connect-status # Flush much faster than normal, which is useful for testing/debugging -offset.flush.interval.ms=10000 \ No newline at end of file +offset.flush.interval.ms=10000 + +# These are provided to inform the user about the presence of the REST host and port configs +# Hostname & Port for the REST API to listen on. If this is set, it will bind to the interface used to listen to requests. +#rest.host.name= +#rest.port=8083 + +# The Hostname & Port that will be given out to other workers to connect to i.e. URLs that are routable from other servers. +#rest.advertised.host.name= +#rest.advertised.port= From b1272500bd7a80931d9571b426afb6a2de7ddeca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?S=C3=B6nke=20Liebau?= Date: Sun, 5 Mar 2017 10:28:36 -0800 Subject: [PATCH 068/101] =?UTF-8?q?KAFKA-4567=20-=20Connect=20Producer=20a?= =?UTF-8?q?nd=20Consumer=20ignore=20ssl=20parameters=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit KAFKA-4567 - Connect Producer and Consumer ignore ssl parameters configured for worker Added brief explanation to the docs about parameter inheritance of Kafka consumers and producers from the worker config. Author: Sönke Liebau Reviewers: Gwen Shapira Closes #2511 from soenkeliebau/KAFKA-4567 --- docs/connect.html | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/connect.html b/docs/connect.html index 1af5ed95f6051..d6b6f0065873e 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -56,6 +56,8 @@

    Running Kafka Connectoffset.storage.file.filename - File to store offset data in + The parameters that are configured here are intended for producers and consumers used by Kafka Connect to access the configuration, offset and status topics. For configuration of Kafka source and Kafka sink tasks, the same parameters can be used but need to be prefixed with consumer. and producer. respectively. The only parameter that is inherited from the worker configuration is bootstrap.servers, which in most cases will be sufficient, since the same cluster is often used for all purposes. A notable exeption is a secured cluster, which requires extra parameters to allow connections. These parameters will need to be set up to three times in the worker configuration, once for management access, once for Kafka sinks and once for Kafka sources. + The remaining parameters are connector configuration files. You may include as many as you want, but all will execute within the same process (on different threads). Distributed mode handles automatic balancing of work, allows you to scale up (or down) dynamically, and offers fault tolerance both in the active tasks and for configuration and offset commit data. Execution is very similar to standalone mode: From 5b013d9cd28d86b50109e8e08f813c3b5af0054b Mon Sep 17 00:00:00 2001 From: Hamidreza Afzali Date: Sun, 5 Mar 2017 22:42:16 -0800 Subject: [PATCH 069/101] KAFKA-4828: ProcessorTopologyTestDriver does not work when using through This resolves the following issues in the ProcessorTopologyTestDriver: - It should not create an internal changelog topic when using `through()` and `table()` - It should forward the produced record back into the topology if it is to a source topic Jira ticket: https://issues.apache.org/jira/browse/KAFKA-4828 The contribution is my original work and I license the work to the project under the project's open source license. Author: Hamidreza Afzali Reviewers: Matthias J. Sax, Guozhang Wang Closes #2629 from hrafzali/KAFKA-4828_ProcessorTopologyTestDriver_through --- .../processor/TopologyBuilderTest.java | 2 +- .../internals/ProcessorTopologyTest.java | 22 ++++++++- .../test/ProcessorTopologyTestDriver.java | 46 +++++++++---------- 3 files changed, 43 insertions(+), 27 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java index c3474b0934482..88a420a75b574 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -629,7 +629,7 @@ public void shouldThroughOnUnassignedStateStoreAccess() { goodNodeName) .addProcessor(badNodeName, new LocalMockProcessorSupplier(), sourceNodeName); - final ProcessorTopologyTestDriver driver = new ProcessorTopologyTestDriver(streamsConfig, builder, LocalMockProcessorSupplier.STORE_NAME); + final ProcessorTopologyTestDriver driver = new ProcessorTopologyTestDriver(streamsConfig, builder); driver.process("topic", null, null); } catch (final StreamsException e) { final Throwable cause = e.getCause(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index 322c17810c206..65b3e2f67aea8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -189,7 +189,7 @@ public void testDrivingMultiplexByNameTopology() { @Test public void testDrivingStatefulTopology() { String storeName = "entries"; - driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName), storeName); + driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName)); driver.process(INPUT_TOPIC_1, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); driver.process(INPUT_TOPIC_1, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); driver.process(INPUT_TOPIC_1, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); @@ -214,7 +214,7 @@ public void shouldDriveGlobalStore() throws Exception { final TopologyBuilder topologyBuilder = this.builder .addGlobalStore(globalStore, global, STRING_DESERIALIZER, STRING_DESERIALIZER, topic, "processor", define(new StatefulProcessor("my-store"))); - driver = new ProcessorTopologyTestDriver(config, topologyBuilder, "my-store"); + driver = new ProcessorTopologyTestDriver(config, topologyBuilder); driver.process(topic, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); driver.process(topic, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); assertEquals("value1", globalStore.get("key1")); @@ -235,6 +235,17 @@ public void testDrivingSimpleMultiSourceTopology() { assertNoOutputRecord(OUTPUT_TOPIC_1); } + @Test + public void testDrivingForwardToSourceTopology() { + driver = new ProcessorTopologyTestDriver(config, createForwardToSourceTopology()); + driver.process(INPUT_TOPIC_1, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC_1, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC_1, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3"); + } + @Test public void testDrivingInternalRepartitioningTopology() { driver = new ProcessorTopologyTestDriver(config, createInternalRepartitioningTopology()); @@ -380,6 +391,13 @@ private TopologyBuilder createInternalRepartitioningWithValueTimestampTopology() .addSink("sink1", OUTPUT_TOPIC_1, "source1"); } + private TopologyBuilder createForwardToSourceTopology() { + return builder.addSource("source-1", INPUT_TOPIC_1) + .addSink("sink-1", OUTPUT_TOPIC_1, "source-1") + .addSource("source-2", OUTPUT_TOPIC_1) + .addSink("sink-2", OUTPUT_TOPIC_2, "source-2"); + } + private TopologyBuilder createSimpleMultiSourceTopology(int partition) { return builder.addSource("source-1", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC_1) .addProcessor("processor-1", define(new ForwardingProcessor()), "source-1") diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index b704aa70e3623..1e97e11b41b5d 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -52,7 +52,6 @@ import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; -import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.RecordCollectorImpl; import org.apache.kafka.streams.processor.internals.StateDirectory; @@ -143,9 +142,10 @@ public class ProcessorTopologyTestDriver { private final Serializer bytesSerializer = new ByteArraySerializer(); - private final String applicationId = "test-driver-application"; + private final static String APPLICATION_ID = "test-driver-application"; + private final static int PARTITION_ID = 0; + private final static TaskId TASK_ID = new TaskId(0, PARTITION_ID); - private final TaskId id; private final ProcessorTopology topology; private final MockConsumer consumer; private final MockProducer producer; @@ -163,11 +163,9 @@ public class ProcessorTopologyTestDriver { * Create a new test driver instance. * @param config the stream configuration for the topology * @param builder the topology builder that will be used to create the topology instance - * @param storeNames the optional names of the state stores that are used by the topology */ - public ProcessorTopologyTestDriver(StreamsConfig config, TopologyBuilder builder, String... storeNames) { - id = new TaskId(0, 0); - topology = builder.setApplicationId(applicationId).build(null); + public ProcessorTopologyTestDriver(StreamsConfig config, TopologyBuilder builder) { + topology = builder.setApplicationId(APPLICATION_ID).build(null); globalTopology = builder.buildGlobalStateTopology(); // Set up the consumer and producer ... @@ -175,10 +173,10 @@ public ProcessorTopologyTestDriver(StreamsConfig config, TopologyBuilder builder producer = new MockProducer(true, bytesSerializer, bytesSerializer) { @Override public List partitionsFor(String topic) { - return Collections.singletonList(new PartitionInfo(topic, 0, null, null, null)); + return Collections.singletonList(new PartitionInfo(topic, PARTITION_ID, null, null, null)); } }; - restoreStateConsumer = createRestoreConsumer(id, storeNames); + restoreStateConsumer = createRestoreConsumer(TASK_ID, topology.storeToChangelogTopic()); // Identify internal topics for forwarding in process ... for (TopologyBuilder.TopicsInfo topicsInfo : builder.topicGroups().values()) { @@ -187,14 +185,14 @@ public List partitionsFor(String topic) { // Set up all of the topic+partition information and subscribe the consumer to each ... for (String topic : topology.sourceTopics()) { - TopicPartition tp = new TopicPartition(topic, 1); + TopicPartition tp = new TopicPartition(topic, PARTITION_ID); partitionsByTopic.put(topic, tp); offsetsByTopicPartition.put(tp, new AtomicLong()); } consumer.assign(offsetsByTopicPartition.keySet()); - final StateDirectory stateDirectory = new StateDirectory(applicationId, TestUtils.tempDirectory().getPath(), Time.SYSTEM); + final StateDirectory stateDirectory = new StateDirectory(APPLICATION_ID, TestUtils.tempDirectory().getPath(), Time.SYSTEM); final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics()); final ThreadCache cache = new ThreadCache("mock", 1024 * 1024, streamsMetrics); @@ -218,8 +216,8 @@ public List partitionsFor(String topic) { } if (!partitionsByTopic.isEmpty()) { - task = new StreamTask(id, - applicationId, + task = new StreamTask(TASK_ID, + APPLICATION_ID, partitionsByTopic.values(), topology, consumer, @@ -263,8 +261,8 @@ private void process(String topicName, byte[] key, byte[] value, long timestamp) } outputRecords.add(record); - // Forward back into the topology if the produced record is to an internal topic ... - if (internalTopics.contains(record.topic())) { + // Forward back into the topology if the produced record is to an internal or a source topic ... + if (internalTopics.contains(record.topic()) || topology.sourceTopics().contains(record.topic())) { process(record.topic(), record.key(), record.value(), record.timestamp()); } } @@ -339,7 +337,7 @@ private Iterable> records(ConsumerRecord * This is often useful in test cases to pre-populate the store before the test case instructs the topology to @@ -355,7 +353,7 @@ public StateStore getStateStore(String name) { /** * Get the {@link KeyValueStore} with the given name. The name should have been supplied via - * {@link #ProcessorTopologyTestDriver(StreamsConfig, TopologyBuilder, String...) this object's constructor}, and is + * {@link #ProcessorTopologyTestDriver(StreamsConfig, TopologyBuilder) this object's constructor}, and is * presumed to be used by a Processor within the topology. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to @@ -393,10 +391,10 @@ public void close() { * driver object unless this method is overwritten with a functional consumer. * * @param id the ID of the stream task - * @param storeNames the names of the stores that this + * @param storeToChangelogTopic the map of the names of the stores to the changelog topics * @return the mock consumer; never null */ - protected MockConsumer createRestoreConsumer(TaskId id, String... storeNames) { + protected MockConsumer createRestoreConsumer(TaskId id, Map storeToChangelogTopic) { MockConsumer consumer = new MockConsumer(OffsetResetStrategy.LATEST) { @Override public synchronized void seekToEnd(Collection partitions) { @@ -414,16 +412,16 @@ public synchronized long position(TopicPartition partition) { return 0L; } }; - // For each store name ... - for (String storeName : storeNames) { - String topicName = ProcessorStateManager.storeChangelogTopic(applicationId, storeName); + // For each store ... + for (Map.Entry storeAndTopic: storeToChangelogTopic.entrySet()) { + String topicName = storeAndTopic.getValue(); // Set up the restore-state topic ... // consumer.subscribe(new TopicPartition(topicName, 1)); // Set up the partition that matches the ID (which is what ProcessorStateManager expects) ... List partitionInfos = new ArrayList<>(); - partitionInfos.add(new PartitionInfo(topicName, id.partition, null, null, null)); + partitionInfos.add(new PartitionInfo(topicName, PARTITION_ID, null, null, null)); consumer.updatePartitions(topicName, partitionInfos); - consumer.updateEndOffsets(Collections.singletonMap(new TopicPartition(topicName, id.partition), 0L)); + consumer.updateEndOffsets(Collections.singletonMap(new TopicPartition(topicName, PARTITION_ID), 0L)); } return consumer; } From f111f2a7167f3abcb0b0e53ac22f9f7bb367766e Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Mon, 6 Mar 2017 10:46:04 +0000 Subject: [PATCH 070/101] MINOR: additional refactoring around the use of Errors A couple of updates were missed in the [PR](https://github.com/apache/kafka/pull/2475) that replaced the use of error codes with Errors objects. Author: Vahid Hashemian Reviewers: Ismael Juma Closes #2635 from vahidhashemian/minor/Errors_refactoring_leftover --- .../kafka/common/OffsetMetadataAndError.scala | 2 +- .../kafka/coordinator/GroupCoordinator.scala | 4 +-- .../kafka/server/ReplicaFetcherThread.scala | 2 +- .../kafka/api/AuthorizerIntegrationTest.scala | 30 +++++++++---------- 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 46c088104780e..e0aa46dedbf8d 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -58,7 +58,7 @@ case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Errors def metadata = offsetMetadata.metadata - override def toString = "[%s,ErrorCode %d]".format(offsetMetadata, error) + override def toString = "[%s, Error=%s]".format(offsetMetadata, error) } object OffsetMetadataAndError { diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 891896a705078..1735dc87a662c 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -474,8 +474,8 @@ class GroupCoordinator(val brokerId: Int, if (!isActive.get) { (Errors.GROUP_COORDINATOR_NOT_AVAILABLE, List[GroupOverview]()) } else { - val errorCode = if (groupManager.isLoading()) Errors.GROUP_LOAD_IN_PROGRESS else Errors.NONE - (errorCode, groupManager.currentGroups.map(_.overview).toList) + val error = if (groupManager.isLoading()) Errors.GROUP_LOAD_IN_PROGRESS else Errors.NONE + (error, groupManager.currentGroups.map(_.overview).toList) } } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 7fb02a30b549a..3b6adec21fefc 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -278,7 +278,7 @@ class ReplicaFetcherThread(name: String, partitionData.offset else partitionData.offsets.get(0) - case errorCode => throw errorCode.exception + case error => throw error.exception } } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 3368c090e2a44..22efa1f124165 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -291,20 +291,20 @@ class AuthorizerIntegrationTest extends BaseRequestTest { for ((key, request) <- requestKeyToRequest) { removeAllAcls val resources = RequestKeysToAcls(key).map(_._1.resourceType).toSet - sendRequestAndVerifyResponseErrorCode(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false) + sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false) val resourceToAcls = RequestKeysToAcls(key) resourceToAcls.get(topicResource).map { acls => val describeAcls = TopicDescribeAcl(topicResource) val isAuthorized = describeAcls == acls addAndVerifyAcls(describeAcls, topicResource) - sendRequestAndVerifyResponseErrorCode(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true) + sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true) removeAllAcls } for ((resource, acls) <- resourceToAcls) addAndVerifyAcls(acls, resource) - sendRequestAndVerifyResponseErrorCode(key, request, resources, isAuthorized = true, isAuthorizedTopicDescribe = false) + sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = true, isAuthorizedTopicDescribe = false) } } @@ -330,20 +330,20 @@ class AuthorizerIntegrationTest extends BaseRequestTest { for ((key, request) <- requestKeyToRequest) { removeAllAcls val resources = RequestKeysToAcls(key).map(_._1.resourceType).toSet - sendRequestAndVerifyResponseErrorCode(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false, topicExists = false) + sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false, topicExists = false) val resourceToAcls = RequestKeysToAcls(key) resourceToAcls.get(topicResource).map { acls => val describeAcls = TopicDescribeAcl(topicResource) val isAuthorized = describeAcls == acls addAndVerifyAcls(describeAcls, topicResource) - sendRequestAndVerifyResponseErrorCode(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true, topicExists = false) + sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true, topicExists = false) removeAllAcls } for ((resource, acls) <- resourceToAcls) addAndVerifyAcls(acls, resource) - sendRequestAndVerifyResponseErrorCode(key, request, resources, isAuthorized = true, isAuthorizedTopicDescribe = false, topicExists = false) + sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = true, isAuthorizedTopicDescribe = false, topicExists = false) } } @@ -805,18 +805,18 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } - def sendRequestAndVerifyResponseErrorCode(apiKey: ApiKeys, - request: AbstractRequest, - resources: Set[ResourceType], - isAuthorized: Boolean, - isAuthorizedTopicDescribe: Boolean, - topicExists: Boolean = true): AbstractResponse = { + def sendRequestAndVerifyResponseError(apiKey: ApiKeys, + request: AbstractRequest, + resources: Set[ResourceType], + isAuthorized: Boolean, + isAuthorizedTopicDescribe: Boolean, + topicExists: Boolean = true): AbstractResponse = { val resp = connectAndSend(request, apiKey) val response = RequestKeyToResponseDeserializer(apiKey).getMethod("parse", classOf[ByteBuffer], classOf[Short]).invoke( null, resp, request.version: java.lang.Short).asInstanceOf[AbstractResponse] val error = RequestKeyToError(apiKey).asInstanceOf[(AbstractResponse) => Errors](response) - val authorizationErrorCodes = resources.flatMap { resourceType => + val authorizationErrors = resources.flatMap { resourceType => if (resourceType == Topic) { if (isAuthorized) Set(Errors.UNKNOWN_TOPIC_OR_PARTITION, Topic.error) @@ -831,9 +831,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { if (topicExists) if (isAuthorized) - assertFalse(s"$apiKey should be allowed. Found unexpected authorization error $error", authorizationErrorCodes.contains(error)) + assertFalse(s"$apiKey should be allowed. Found unexpected authorization error $error", authorizationErrors.contains(error)) else - assertTrue(s"$apiKey should be forbidden. Found error $error but expected one of $authorizationErrorCodes", authorizationErrorCodes.contains(error)) + assertTrue(s"$apiKey should be forbidden. Found error $error but expected one of $authorizationErrors", authorizationErrors.contains(error)) else if (resources == Set(Topic)) assertEquals(s"$apiKey had an unexpected error", Errors.UNKNOWN_TOPIC_OR_PARTITION, error) else From 79f85039d7be1b4266f06715a487f3635558ded6 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 6 Mar 2017 10:47:36 +0000 Subject: [PATCH 071/101] KAFKA-3989; Initial support for adding a JMH benchmarking module Author: bbejeck Reviewers: Ewen Cheslack-Postava , Ismael Juma Closes #1712 from bbejeck/KAFKA-3989_create_jmh_benchmarking_module --- build.gradle | 43 ++++++++++++ checkstyle/import-control.xml | 9 +++ jmh-benchmarks/README.md | 61 +++++++++++++++++ jmh-benchmarks/jmh.sh | 42 ++++++++++++ .../kafka/jmh/cache/LRUCacheBenchmark.java | 68 +++++++++++++++++++ settings.gradle | 2 +- 6 files changed, 224 insertions(+), 1 deletion(-) create mode 100644 jmh-benchmarks/README.md create mode 100755 jmh-benchmarks/jmh.sh create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java diff --git a/build.gradle b/build.gradle index caac99d1e25c9..57beebeed2153 100644 --- a/build.gradle +++ b/build.gradle @@ -27,6 +27,7 @@ buildscript { classpath "org.ajoberstar:grgit:1.7.0" classpath 'com.github.ben-manes:gradle-versions-plugin:0.13.0' classpath 'org.scoverage:gradle-scoverage:2.1.0' + classpath 'com.github.jengelman.gradle.plugins:shadow:1.2.4' } } @@ -822,6 +823,48 @@ project(':streams:examples') { } } +project(':jmh-benchmarks') { + + apply plugin: 'com.github.johnrengelman.shadow' + + shadowJar { + baseName = 'kafka-jmh-benchmarks-all' + classifier = null + version = null + } + + dependencies { + compile project(':clients') + compile project(':streams') + compile 'org.openjdk.jmh:jmh-core:1.17.5' + compile 'org.openjdk.jmh:jmh-generator-annprocess:1.17.5' + compile 'org.openjdk.jmh:jmh-core-benchmarks:1.17.5' + } + + jar { + manifest { + attributes "Main-Class": "org.openjdk.jmh.Main" + } + } + + + task jmh(type: JavaExec, dependsOn: [':jmh-benchmarks:clean', ':jmh-benchmarks:shadowJar']) { + + main="-jar" + + doFirst { + if (System.getProperty("jmhArgs")) { + args System.getProperty("jmhArgs").split(',') + } + args = [shadowJar.archivePath, *args] + } + } + + javadoc { + enabled = false + } +} + project(':log4j-appender') { archivesBaseName = "kafka-log4j-appender" diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 6c72e63c1e249..fa98593b0ff26 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -181,6 +181,15 @@ + + + + + + + + + diff --git a/jmh-benchmarks/README.md b/jmh-benchmarks/README.md new file mode 100644 index 0000000000000..53807ea06ff5b --- /dev/null +++ b/jmh-benchmarks/README.md @@ -0,0 +1,61 @@ +###JMH-Benchmark module + +This module contains benchmarks written using [JMH](http://openjdk.java.net/projects/code-tools/jmh/) from OpenJDK. +Writing correct micro-benchmarks is Java (or another JVM language) is difficult and there are many non-obvious pitfalls (many +due to compiler optimizations). JMH is a framework for running and analyzing benchmarks (micro or macro) written in Java (or +another JVM language). + +For help in writing correct JMH tests, the best place to start is the [sample code](http://hg.openjdk.java.net/code-tools/jmh/file/tip/jmh-samples/src/main/java/org/openjdk/jmh/samples/) provided +by the JMH project. + +Typically, JMH is expected to run as a separate project in Maven. The jmh-benchmarks module uses +the [gradle shadow jar](https://github.com/johnrengelman/shadow) plugin to emulate this behavior, by creating the required +uber-jar file containing the benchmarking code and required JMH classes. + +JMH is highly configurable and users are encouraged to look through the samples for suggestions +on what options are available. A good tutorial for using JMH can be found [here](http://tutorials.jenkov.com/java-performance/jmh.html#return-value-from-benchmark-method) + +###Gradle Tasks / Running benchmarks in gradle + +If no benchmark mode is specified, the default is used which is throughput. It is assumed that users run +the gradle tasks with './gradlew' from the root of the Kafka project. + +* jmh-benchmarks:shadowJar - creates the uber jar required to run the benchmarks. + +* jmh-benchmarks:jmh - runs the `clean` and `shadowJar` tasks followed by all the benchmarks. + +### Using the jmh script +If you want to set specific JMH flags or only run a certain test(s) passing arguments via +gradle tasks is cumbersome. Instead you can use the `jhm.sh` script. NOTE: It is assumed users run +the jmh.sh script from the jmh-benchmarks module. + +* Run a specific test setting fork-mode (number iterations) to 2 :`./jmh.sh -f 2 LRUCacheBenchmark` + +* By default all JMH output goes to stdout. To run a benchmark and capture the results in a file: +`./jmh.sh -f 2 -o benchmarkResults.txt LRUCacheBenchmark` +NOTE: For now this script needs to be run from the jmh-benchmarks directory. + +### Running JMH outside of gradle +The JMH benchmarks can be run outside of gradle as you would with any executable jar file: +`java -jar /jmh-benchmarks/build/libs/kafka-jmh-benchmarks-all.jar -f2 LRUCacheBenchmark` + +### JMH Options +Some common JMH options are: +```text + + -e Benchmarks to exclude from the run. + + -f How many times to fork a single benchmark. Use 0 to + disable forking altogether. Warning: disabling + forking may have detrimental impact on benchmark + and infrastructure reliability, you might want + to use different warmup mode instead. + + -o Redirect human-readable output to a given file. + + + + -v Verbosity mode. Available modes are: [SILENT, NORMAL, + EXTRA] +``` +To view all options run jmh with the -h flag. diff --git a/jmh-benchmarks/jmh.sh b/jmh-benchmarks/jmh.sh new file mode 100755 index 0000000000000..7f3927aa84332 --- /dev/null +++ b/jmh-benchmarks/jmh.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# 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. + +base_dir=$(dirname $0) +jmh_project_name="jmh-benchmarks" + +if [ ${base_dir} == "." ]; then + gradlew_dir=".." +elif [ ${base_dir} == ${jmh_project_name} ]; then + gradlew_dir="." +else + echo "JMH Benchmarks need to be run from the root of the kafka repository or the 'jmh-benchmarks' directory" + exit +fi + +gradleCmd="${gradlew_dir}/gradlew" +libDir="${base_dir}/build/libs" + +echo "running gradlew :jmh-benchmarks:clean :jmh-benchmarks:shadowJar in quiet mode" + +$gradleCmd -q :jmh-benchmarks:clean :jmh-benchmarks:shadowJar + +echo "gradle build done" + +echo "running JMH with args [$@]" + +java -jar ${libDir}/kafka-jmh-benchmarks-all.jar "$@" + +echo "JMH benchmarks done" diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java new file mode 100644 index 0000000000000..ecf73f9ab597d --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.jmh.cache; + +import org.apache.kafka.common.cache.LRUCache; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +/** + * This is a simple example of a JMH benchmark. + * + * The sample code provided by the JMH project is a great place to start learning how to write correct benchmarks: + * http://hg.openjdk.java.net/code-tools/jmh/file/tip/jmh-samples/src/main/java/org/openjdk/jmh/samples/ + */ +@State(Scope.Thread) +public class LRUCacheBenchmark { + + private LRUCache lruCache; + + private final String key = "the_key_to_use"; + private final String value = "the quick brown fox jumped over the lazy dog the olympics are about to start"; + int counter; + + + @Setup(Level.Trial) + public void setUpCaches() { + lruCache = new LRUCache<>(100); + } + + @Benchmark + public String testCachePerformance() { + counter++; + lruCache.put(key + counter, value + counter); + return lruCache.get(key + counter); + } + + public static void main(String[] args) throws RunnerException { + Options opt = new OptionsBuilder() + .include(LRUCacheBenchmark.class.getSimpleName()) + .forks(2) + .build(); + + new Runner(opt).run(); + } + +} diff --git a/settings.gradle b/settings.gradle index 29d38950a8acb..f0fdf07128c43 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,4 +14,4 @@ // limitations under the License. include 'core', 'examples', 'clients', 'tools', 'streams', 'streams:examples', 'log4j-appender', - 'connect:api', 'connect:transforms', 'connect:runtime', 'connect:json', 'connect:file' + 'connect:api', 'connect:transforms', 'connect:runtime', 'connect:json', 'connect:file', 'jmh-benchmarks' From 63010cbfe5d07df5db060888afd348fab5cbe62c Mon Sep 17 00:00:00 2001 From: Ben Stopford Date: Mon, 6 Mar 2017 12:42:43 +0000 Subject: [PATCH 072/101] KAFKA-4266; ReassignPartitionsClusterTest: ensure ZK publication is completed before start Increase the reliability of the one temporal comparison in ReassignPartitionsClusterTest by imposing a delay after ZK is updated. This should be more reliable than just increasing the amount of data. This relates to a previous PR: https://github.com/apache/kafka/pull/1982 Author: Ben Stopford Reviewers: Ismael Juma Closes #1997 from benstopford/KAFKA-4266 --- .../admin/ReassignPartitionsCommand.scala | 37 ++++++++----- .../kafka/ReplicationQuotasTestRig.scala | 3 +- .../admin/ReassignPartitionsClusterTest.scala | 55 +++++++++++-------- .../admin/ReassignPartitionsCommandTest.scala | 7 ++- .../kafka/server/ReplicationQuotasTest.scala | 1 - 5 files changed, 60 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 4e7b4e0c3937c..c167633f882b5 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -17,8 +17,9 @@ package kafka.admin import joptsimple.OptionParser -import kafka.server.{DynamicConfig, ConfigType} +import kafka.server.{ConfigType, DynamicConfig} import kafka.utils._ + import scala.collection._ import org.I0Itec.zkclient.exception.ZkNodeExistsException import kafka.common.{AdminCommandFailedException, TopicAndPartition} @@ -29,6 +30,10 @@ import org.apache.kafka.common.security.JaasUtils object ReassignPartitionsCommand extends Logging { + case class Throttle(value: Long, postUpdateAction: () => Unit = () => ()) + + private[admin] val NoThrottle = Throttle(-1) + def main(args: Array[String]): Unit = { val opts = validateAndParseArgs(args) @@ -146,10 +151,10 @@ object ReassignPartitionsCommand extends Logging { val reassignmentJsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt) val reassignmentJsonString = Utils.readFileAsString(reassignmentJsonFile) val throttle = if (opts.options.has(opts.throttleOpt)) opts.options.valueOf(opts.throttleOpt) else -1 - executeAssignment(zkUtils, reassignmentJsonString, throttle) + executeAssignment(zkUtils, reassignmentJsonString, Throttle(throttle)) } - def executeAssignment(zkUtils: ZkUtils, reassignmentJsonString: String, throttle: Long = -1) { + def executeAssignment(zkUtils: ZkUtils, reassignmentJsonString: String, throttle: Throttle) { val partitionsToBeReassigned = parseAndValidate(zkUtils, reassignmentJsonString) val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, partitionsToBeReassigned.toMap) @@ -160,7 +165,7 @@ object ReassignPartitionsCommand extends Logging { } else { printCurrentAssignment(zkUtils, partitionsToBeReassigned) - if (throttle >= 0) + if (throttle.value >= 0) println(String.format("Warning: You must run Verify periodically, until the reassignment completes, to ensure the throttle is removed. You can also alter the throttle by rerunning the Execute command passing a new value.")) if (reassignPartitionsCommand.reassignPartitions(throttle)) { println("Successfully started reassignment of partitions.") @@ -178,7 +183,7 @@ object ReassignPartitionsCommand extends Logging { def parseAndValidate(zkUtils: ZkUtils, reassignmentJsonString: String): Seq[(TopicAndPartition, Seq[Int])] = { val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentDataWithoutDedup(reassignmentJsonString) - + if (partitionsToBeReassigned.isEmpty) throw new AdminCommandFailedException("Partition reassignment data file is empty") val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map { case (tp, _) => tp }) @@ -306,15 +311,19 @@ object ReassignPartitionsCommand extends Logging { class ReassignPartitionsCommand(zkUtils: ZkUtils, proposedAssignment: Map[TopicAndPartition, Seq[Int]], admin: AdminUtilities = AdminUtils) extends Logging { + import ReassignPartitionsCommand._ + def existingAssignment(): Map[TopicAndPartition, Seq[Int]] = { val proposedTopics = proposedAssignment.keySet.map(_.topic).toSeq zkUtils.getReplicaAssignmentForTopics(proposedTopics) } - private def maybeThrottle(throttle: Long): Unit = { - if (throttle >= 0) { - maybeLimit(throttle) + private def maybeThrottle(throttle: Throttle): Unit = { + if (throttle.value >= 0) { assignThrottledReplicas(existingAssignment(), proposedAssignment) + maybeLimit(throttle) + throttle.postUpdateAction() + println(s"The throttle limit was set to ${throttle.value} B/s") } } @@ -322,19 +331,18 @@ class ReassignPartitionsCommand(zkUtils: ZkUtils, proposedAssignment: Map[TopicA * Limit the throttle on currently moving replicas. Note that this command can use used to alter the throttle, but * it may not alter all limits originally set, if some of the brokers have completed their rebalance. */ - def maybeLimit(throttle: Long) { - if (throttle >= 0) { + def maybeLimit(throttle: Throttle) { + if (throttle.value >= 0) { val existingBrokers = existingAssignment().values.flatten.toSeq val proposedBrokers = proposedAssignment.values.flatten.toSeq val brokers = (existingBrokers ++ proposedBrokers).distinct for (id <- brokers) { val configs = admin.fetchEntityConfig(zkUtils, ConfigType.Broker, id.toString) - configs.put(DynamicConfig.Broker.LeaderReplicationThrottledRateProp, throttle.toString) - configs.put(DynamicConfig.Broker.FollowerReplicationThrottledRateProp, throttle.toString) + configs.put(DynamicConfig.Broker.LeaderReplicationThrottledRateProp, throttle.value.toString) + configs.put(DynamicConfig.Broker.FollowerReplicationThrottledRateProp, throttle.value.toString) admin.changeBrokerConfig(zkUtils, Seq(id), configs) } - println(s"The throttle limit was set to $throttle B/s") } } @@ -384,7 +392,7 @@ class ReassignPartitionsCommand(zkUtils: ZkUtils, proposedAssignment: Map[TopicA allProposed.filter { case (tp, _) => tp.topic == topic }) } - def reassignPartitions(throttle: Long = -1): Boolean = { + def reassignPartitions(throttle: Throttle = NoThrottle): Boolean = { maybeThrottle(throttle) try { val validPartitions = proposedAssignment.filter { case (p, _) => validatePartition(zkUtils, p.topic, p.partition) } @@ -422,6 +430,7 @@ class ReassignPartitionsCommand(zkUtils: ZkUtils, proposedAssignment: Map[TopicA } } + sealed trait ReassignmentStatus { def status: Int } case object ReassignmentCompleted extends ReassignmentStatus { val status = 1 } case object ReassignmentInProgress extends ReassignmentStatus { val status = 0 } diff --git a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala index 71a5091f5df2b..38d07ba7c881b 100644 --- a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala +++ b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala @@ -21,6 +21,7 @@ import java.io.{File, FileOutputStream, PrintWriter} import javax.imageio.ImageIO import kafka.admin.ReassignPartitionsCommand +import kafka.admin.ReassignPartitionsCommand.Throttle import kafka.common.TopicAndPartition import org.apache.kafka.common.TopicPartition import kafka.server.{KafkaConfig, KafkaServer, QuotaType} @@ -139,7 +140,7 @@ object ReplicationQuotasTestRig { val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, brokers, json(topicName), true)._1 val start = System.currentTimeMillis() - ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), config.throttle) + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), Throttle(config.throttle)) //Await completion waitForReassignmentToComplete() diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala index c211c245eb18c..c576a5c1c2f47 100644 --- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala @@ -12,6 +12,7 @@ */ package kafka.admin +import kafka.admin.ReassignPartitionsCommand._ import kafka.common.{AdminCommandFailedException, TopicAndPartition} import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.TestUtils._ @@ -21,14 +22,15 @@ import kafka.zk.ZooKeeperTestHarness import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Before, Test} import kafka.admin.ReplicationQuotaUtils._ - -import scala.collection.{Map, Seq} - +import scala.collection.Map +import scala.collection.Seq class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { val partitionId = 0 var servers: Seq[KafkaServer] = null val topicName = "my-topic" + val delayMs = 1000 + def zkUpdateDelay = {Thread.sleep(delayMs)} @Before override def setUp() { @@ -55,7 +57,8 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { createTopic(zkUtils, topicName, Map(partition -> Seq(100)), servers = servers) //When we move the replica on 100 to broker 101 - ReassignPartitionsCommand.executeAssignment(zkUtils, s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101]}]}""") + val topicJson: String = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101]}]}""" + ReassignPartitionsCommand.executeAssignment(zkUtils, topicJson, NoThrottle) waitForReassignmentToComplete() //Then the replica should be on 101 @@ -74,8 +77,8 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { ), servers = servers) //When rebalancing - val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, brokers, json(topicName), true)._1 - ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment)) + val newAssignment = generateAssignment(zkUtils, brokers, json(topicName), true)._1 + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), NoThrottle) waitForReassignmentToComplete() //Then the replicas should span all three brokers @@ -95,8 +98,8 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { ), servers = servers) //When rebalancing - val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, Array(100, 101), json(topicName), true)._1 - ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment)) + val newAssignment = generateAssignment(zkUtils, Array(100, 101), json(topicName), true)._1 + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), NoThrottle) waitForReassignmentToComplete() //Then replicas should only span the first two brokers @@ -127,7 +130,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { ) //When rebalancing - ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(proposed)) + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(proposed), NoThrottle) waitForReassignmentToComplete() //Then the proposed changes should have been made @@ -142,6 +145,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { @Test def shouldExecuteThrottledReassignment() { + //Given partitions on 3 of 3 brokers val brokers = Array(100, 101, 102) startBrokers(brokers) @@ -150,33 +154,35 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { ), servers = servers) //Given throttle set so replication will take a certain number of secs - val initialThrottle: Long = 10 * 1000 * 1000 + val initialThrottle = Throttle(10 * 1000 * 1000, () => zkUpdateDelay) val expectedDurationSecs = 5 val numMessages: Int = 500 val msgSize: Int = 100 * 1000 produceMessages(servers, topicName, numMessages, acks = 0, msgSize) - assertEquals(expectedDurationSecs, numMessages * msgSize / initialThrottle) + assertEquals(expectedDurationSecs, numMessages * msgSize / initialThrottle.value) //Start rebalance which will move replica on 100 -> replica on 102 - val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, Array(101, 102), json(topicName), true)._1 + val newAssignment = generateAssignment(zkUtils, Array(101, 102), json(topicName), true)._1 val start = System.currentTimeMillis() ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), initialThrottle) //Check throttle config. Should be throttling replica 0 on 100 and 102 only. - checkThrottleConfigAddedToZK(initialThrottle, servers, topicName, "0:100,0:101", "0:102") + checkThrottleConfigAddedToZK(initialThrottle.value, servers, topicName, "0:100,0:101", "0:102") //Await completion waitForReassignmentToComplete() - val took = System.currentTimeMillis() - start + val took = System.currentTimeMillis() - start - delayMs //Check move occurred val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName) assertEquals(Seq(101, 102), actual.values.flatten.toSeq.distinct.sorted) //Then command should have taken longer than the throttle rate - assertTrue(s"Expected replication to be > ${expectedDurationSecs * 0.9 * 1000} but was $took", took > expectedDurationSecs * 0.9 * 1000) - assertTrue(s"Expected replication to be < ${expectedDurationSecs * 2 * 1000} but was $took", took < expectedDurationSecs * 2 * 1000) + assertTrue(s"Expected replication to be > ${expectedDurationSecs * 0.9 * 1000} but was $took", + took > expectedDurationSecs * 0.9 * 1000) + assertTrue(s"Expected replication to be < ${expectedDurationSecs * 2 * 1000} but was $took", + took < expectedDurationSecs * 2 * 1000) } @@ -211,7 +217,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { TopicAndPartition("topic1", 2) -> Seq(103, 104), //didn't move TopicAndPartition("topic2", 2) -> Seq(103, 104) //didn't move ) - ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), throttle) + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), Throttle(throttle)) //Check throttle config. Should be throttling specific replicas for each topic. checkThrottleConfigAddedToZK(throttle, servers, "topic1", @@ -238,15 +244,15 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { produceMessages(servers, topicName, numMessages = 200, acks = 0, valueBytes = 100 * 1000) //Start rebalance - val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, Array(101, 102), json(topicName), true)._1 + val newAssignment = generateAssignment(zkUtils, Array(101, 102), json(topicName), true)._1 - ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), initialThrottle) + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), Throttle(initialThrottle)) //Check throttle config checkThrottleConfigAddedToZK(initialThrottle, servers, topicName, "0:100,0:101", "0:102") //Ensure that running Verify, whilst the command is executing, should have no effect - ReassignPartitionsCommand.verifyAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment)) + verifyAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment)) //Check throttle config again checkThrottleConfigAddedToZK(initialThrottle, servers, topicName, "0:100,0:101", "0:102") @@ -254,7 +260,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { //Now re-run the same assignment with a larger throttle, which should only act to increase the throttle and make progress val newThrottle = initialThrottle * 1000 - ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), newThrottle) + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment), Throttle(newThrottle)) //Check throttle was changed checkThrottleConfigAddedToZK(newThrottle, servers, topicName, "0:100,0:101", "0:102") @@ -263,7 +269,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { waitForReassignmentToComplete() //Verify should remove the throttle - ReassignPartitionsCommand.verifyAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment)) + verifyAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(newAssignment)) //Check removed checkThrottleConfigRemovedFromZK(topicName, servers) @@ -280,12 +286,13 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { createTopic(zkUtils, topicName, Map(0 -> Seq(100)), servers = servers) //When we execute an assignment that includes an invalid partition (1:101 in this case) - ReassignPartitionsCommand.executeAssignment(zkUtils, s"""{"version":1,"partitions":[{"topic":"$topicName","partition":1,"replicas":[101]}]}""") + val topicJson = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":1,"replicas":[101]}]}""" + ReassignPartitionsCommand.executeAssignment(zkUtils, topicJson, NoThrottle) } @Test def shouldPerformThrottledReassignmentOverVariousTopics() { - val throttle = 1000L + val throttle = Throttle(1000L) //Given four brokers servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(conf => TestUtils.createServer(KafkaConfig.fromProps(conf))) diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala index 5ecc19bde25be..9e2398355dd97 100644 --- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala @@ -18,6 +18,7 @@ package kafka.admin import java.util.Properties +import kafka.admin.ReassignPartitionsCommand.Throttle import kafka.common.TopicAndPartition import kafka.log.LogConfig import kafka.log.LogConfig._ @@ -248,7 +249,7 @@ class ReassignPartitionsCommandTest extends Logging { replay(admin) //When - assigner.maybeLimit(1000) + assigner.maybeLimit(Throttle(1000)) //Then for (actual <- propsCapture.getValues) { @@ -282,7 +283,7 @@ class ReassignPartitionsCommandTest extends Logging { replay(admin) //When - assigner.maybeLimit(1000) + assigner.maybeLimit(Throttle(1000)) //Then for (actual <- propsCapture.getValues) { @@ -312,7 +313,7 @@ class ReassignPartitionsCommandTest extends Logging { replay(admin) //When - assigner.maybeLimit(1000) + assigner.maybeLimit(Throttle(1000)) //Then other property remains for (actual <- propsCapture.getValues) { diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 984d3403f0459..b6e36077e6677 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -31,7 +31,6 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.junit.Assert._ import org.junit.{After, Before, Test} - import scala.collection.JavaConverters._ /** From 2064a2a55dde9ae4e263eb7c78da3b4ac2896201 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 6 Mar 2017 09:52:17 -0800 Subject: [PATCH 073/101] HOTFIX: fix broken link for wordcount demo example Author: Matthias J. Sax Reviewers: Guozhang Wang Closes #2644 from mjsax/hotfixBrokerLink --- docs/quickstart.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/quickstart.html b/docs/quickstart.html index 69f6a7a5d4f7a..51d85fd1b6cd3 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -282,7 +282,7 @@

    Step 8: Use

    Kafka Streams is a client library of Kafka for real-time stream processing and analyzing data stored in Kafka brokers. This quickstart example will demonstrate how to run a streaming application coded in this library. Here is the gist -of the WordCountDemo example code (converted to use Java 8 lambda expressions for easy reading). +of the WordCountDemo example code (converted to use Java 8 lambda expressions for easy reading).

     // Serializers/deserializers (serde) for String and Long types
    @@ -446,4 +446,4 @@ 

    Step 8: Use -
    \ No newline at end of file +
    From 5cf491c2765b4912ff5f7f69f0b992a67005b983 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 6 Mar 2017 20:10:45 +0000 Subject: [PATCH 074/101] KAFKA-3155; Avoid long overflow in RecordBatch#maybeExpire `Long.MaxValue` for the linger overflows in `RecordBatch#maybeExpire` when added to the current timestamp. Then causes an error to be set for the batch by `Sender` (not happening every time since it depends on the timing of `Sender`): That error then causes a call to `ProduceRequestResult#done` on the batch, which then makes the check for "not done" fail. Author: Armin Braun Reviewers: Ismael Juma Closes #2639 from original-brownbear/KAFKA-3155 --- .../producer/internals/RecordAccumulator.java | 33 ++++----- .../producer/internals/RecordBatch.java | 57 ++++++++++----- .../clients/producer/internals/Sender.java | 6 +- .../producer/internals/RecordBatchTest.java | 72 +++++++++++++++++++ 4 files changed, 128 insertions(+), 40 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordBatchTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 7ef28e5031006..330682085f3fe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -81,7 +81,7 @@ public final class RecordAccumulator { /** * Create a new record accumulator - * + * * @param batchSize The size to use when allocating {@link MemoryRecords} instances * @param totalSize The maximum memory the record accumulator can use. * @param compression The compression codec for the records @@ -205,7 +205,7 @@ public RecordAppendResult append(TopicPartition tp, // Don't deallocate this buffer in the finally block as it's being used in the record batch buffer = null; - + return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true); } } finally { @@ -284,10 +284,7 @@ public List abortExpiredBatches(int requestTimeout, long now) { * Re-enqueue the given record batch in the accumulator to retry */ public void reenqueue(RecordBatch batch, long now) { - batch.attempts++; - batch.lastAttemptMs = now; - batch.lastAppendTime = now; - batch.setRetry(); + batch.reenqueued(now); Deque deque = getOrCreateDeque(batch.topicPartition); synchronized (deque) { deque.addFirst(batch); @@ -334,16 +331,16 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { } else if (!readyNodes.contains(leader) && !muted.contains(part)) { RecordBatch batch = deque.peekFirst(); if (batch != null) { - boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > nowMs; - long waitedTimeMs = nowMs - batch.lastAttemptMs; + long waitedTimeMs = batch.waitedTimeMs(nowMs); + boolean backingOff = batch.attempts() > 0 && waitedTimeMs < retryBackoffMs; long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs; - long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0); boolean full = deque.size() > 1 || batch.isFull(); boolean expired = waitedTimeMs >= timeToWaitMs; boolean sendable = full || expired || exhausted || closed || flushInProgress(); if (sendable && !backingOff) { readyNodes.add(leader); } else { + long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0); // Note that this results in a conservative estimate since an un-sendable partition may have // a leader that will later be found to have sendable data. However, this is good enough // since we'll just wake up and then sleep again for the remaining time. @@ -374,7 +371,7 @@ public boolean hasUnsent() { /** * Drain all the data for the given nodes and collate them into a list of batches that will fit within the specified * size on a per-node basis. This method attempts to avoid choosing the same topic-node over and over. - * + * * @param cluster The current cluster metadata * @param nodes The list of node to drain * @param maxSize The maximum number of bytes to drain @@ -405,7 +402,7 @@ public Map> drain(Cluster cluster, synchronized (deque) { RecordBatch first = deque.peekFirst(); if (first != null) { - boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now; + boolean backoff = first.attempts() > 0 && first.waitedTimeMs(now) < retryBackoffMs; // Only drain the batch if it is not during backoff period. if (!backoff) { if (size + first.sizeInBytes() > maxSize && !ready.isEmpty()) { @@ -418,7 +415,7 @@ public Map> drain(Cluster cluster, batch.close(); size += batch.sizeInBytes(); ready.add(batch); - batch.drainedMs = now; + batch.drained(now); } } } @@ -458,7 +455,7 @@ public void deallocate(RecordBatch batch) { incomplete.remove(batch); free.deallocate(batch.buffer(), batch.initialCapacity()); } - + /** * Are there any threads currently waiting on a flush? * @@ -472,7 +469,7 @@ boolean flushInProgress() { Map> batches() { return Collections.unmodifiableMap(batches); } - + /** * Initiate the flushing of data from the accumulator...this makes all requests immediately ready */ @@ -578,7 +575,7 @@ public ReadyCheckResult(Set readyNodes, long nextReadyCheckDelayMs, Set(); } - + public void add(RecordBatch batch) { synchronized (incomplete) { this.incomplete.add(batch); } } - + public void remove(RecordBatch batch) { synchronized (incomplete) { boolean removed = this.incomplete.remove(batch); @@ -602,7 +599,7 @@ public void remove(RecordBatch batch) { throw new IllegalStateException("Remove from the incomplete set failed. This should be impossible."); } } - + public Iterable all() { synchronized (incomplete) { return new ArrayList<>(this.incomplete); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index af6262ec63fce..8dacaf5fe9687 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -33,7 +33,7 @@ /** * A batch of records that is or will be sent. - * + * * This class is not thread safe and external synchronization must be used when modifying it */ public final class RecordBatch { @@ -47,12 +47,12 @@ public final class RecordBatch { private final List thunks = new ArrayList<>(); private final MemoryRecordsBuilder recordsBuilder; - volatile int attempts; + private volatile int attempts; int recordCount; int maxRecordSize; - long drainedMs; - long lastAttemptMs; - long lastAppendTime; + private long lastAttemptMs; + private long lastAppendTime; + private long drainedMs; private String expiryErrorMessage; private AtomicBoolean completed; private boolean retry; @@ -69,7 +69,7 @@ public RecordBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long /** * Append the record to the current record set and return the relative offset within that record set - * + * * @return The RecordSend corresponding to this record or null if there isn't sufficient room. */ public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) { @@ -92,7 +92,7 @@ public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, /** * Complete the request. - * + * * @param baseOffset The base offset of the messages assigned by the server * @param logAppendTime The log append time or -1 if CreateTime is being used * @param exception The exception that occurred (or null if the request was successful) @@ -152,13 +152,12 @@ public String toString() { * {@link #expirationDone()} must be invoked to complete the produce future and invoke callbacks. */ public boolean maybeExpire(int requestTimeoutMs, long retryBackoffMs, long now, long lingerMs, boolean isFull) { - if (!this.inRetry() && isFull && requestTimeoutMs < (now - this.lastAppendTime)) expiryErrorMessage = (now - this.lastAppendTime) + " ms has passed since last append"; - else if (!this.inRetry() && requestTimeoutMs < (now - (this.createdMs + lingerMs))) - expiryErrorMessage = (now - (this.createdMs + lingerMs)) + " ms has passed since batch creation plus linger time"; - else if (this.inRetry() && requestTimeoutMs < (now - (this.lastAttemptMs + retryBackoffMs))) - expiryErrorMessage = (now - (this.lastAttemptMs + retryBackoffMs)) + " ms has passed since last attempt plus backoff time"; + else if (!this.inRetry() && requestTimeoutMs < (createdTimeMs(now) - lingerMs)) + expiryErrorMessage = (createdTimeMs(now) - lingerMs) + " ms has passed since batch creation plus linger time"; + else if (this.inRetry() && requestTimeoutMs < (waitedTimeMs(now) - retryBackoffMs)) + expiryErrorMessage = (waitedTimeMs(now) - retryBackoffMs) + " ms has passed since last attempt plus backoff time"; boolean expired = expiryErrorMessage != null; if (expired) @@ -178,6 +177,33 @@ void expirationDone() { new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + ": " + expiryErrorMessage)); } + int attempts() { + return attempts; + } + + void reenqueued(long now) { + attempts++; + lastAttemptMs = Math.max(lastAppendTime, now); + lastAppendTime = Math.max(lastAppendTime, now); + retry = true; + } + + long queueTimeMs() { + return drainedMs - createdMs; + } + + long createdTimeMs(long nowMs) { + return Math.max(0, nowMs - createdMs); + } + + long waitedTimeMs(long nowMs) { + return Math.max(0, nowMs - lastAttemptMs); + } + + void drained(long nowMs) { + this.drainedMs = Math.max(drainedMs, nowMs); + } + /** * Returns if the batch is been retried for sending to kafka */ @@ -185,13 +211,6 @@ private boolean inRetry() { return this.retry; } - /** - * Set retry to true if the batch is being retried (for send) - */ - public void setRetry() { - this.retry = true; - } - public MemoryRecords records() { return recordsBuilder.build(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index b3553afd8ba42..7f27d36845acb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -300,7 +300,7 @@ private void completeBatch(RecordBatch batch, ProduceResponse.PartitionResponse log.warn("Got error produce response with correlation id {} on topic-partition {}, retrying ({} attempts left). Error: {}", correlationId, batch.topicPartition, - this.retries - batch.attempts - 1, + this.retries - batch.attempts() - 1, error); this.accumulator.reenqueue(batch, now); this.sensors.recordRetries(batch.topicPartition.topic(), batch.recordCount); @@ -332,7 +332,7 @@ private void completeBatch(RecordBatch batch, ProduceResponse.PartitionResponse * We can retry a send if the error is transient and the number of attempts taken is fewer than the maximum allowed */ private boolean canRetry(RecordBatch batch, Errors error) { - return batch.attempts < this.retries && error.exception() instanceof RetriableException; + return batch.attempts() < this.retries && error.exception() instanceof RetriableException; } /** @@ -519,7 +519,7 @@ public void updateProduceRequestMetrics(Map> batches) // global metrics this.batchSizeSensor.record(batch.sizeInBytes(), now); - this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now); + this.queueTimeSensor.record(batch.queueTimeMs(), now); this.compressionRateSensor.record(batch.compressionRate()); this.maxRecordSizeSensor.record(batch.maxRecordSize, now); records += batch.recordCount; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordBatchTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordBatchTest.java new file mode 100644 index 0000000000000..6404451b4dff9 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordBatchTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.producer.internals; + +import java.nio.ByteBuffer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.TimestampType; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; + +public class RecordBatchTest { + + private final long now = 1488748346917L; + + private final MemoryRecordsBuilder memoryRecordsBuilder = new MemoryRecordsBuilder(ByteBuffer.allocate(0), + Record.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, 0L, Record.NO_TIMESTAMP, 0); + + /** + * A RecordBatch configured using a very large linger value and a timestamp preceding its create + * time is interpreted correctly as not expired when the linger time is larger than the difference + * between now and create time by RecordBatch#maybeExpire. + */ + @Test + public void testLargeLingerOldNowExpire() { + RecordBatch batch = new RecordBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); + // Set `now` to 2ms before the create time. + assertFalse(batch.maybeExpire(10240, 100L, now - 2L, Long.MAX_VALUE, false)); + } + + /** + * A RecordBatch configured using a very large retryBackoff value with retry = true and a timestamp preceding its + * create time is interpreted correctly as not expired when the retryBackoff time is larger than the difference + * between now and create time by RecordBatch#maybeExpire. + */ + @Test + public void testLargeRetryBackoffOldNowExpire() { + RecordBatch batch = new RecordBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); + // Set batch.retry = true + batch.reenqueued(now); + // Set `now` to 2ms before the create time. + assertFalse(batch.maybeExpire(10240, Long.MAX_VALUE, now - 2L, 10240L, false)); + } + + /** + * A RecordBatch#maybeExpire call with a now value before the create time of the RecordBatch is correctly recognized + * as not expired when invoked with parameter isFull = true. + */ + @Test + public void testLargeFullOldNowExpire() { + RecordBatch batch = new RecordBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); + // Set `now` to 2ms before the create time. + assertFalse(batch.maybeExpire(10240, 10240L, now - 2L, 10240L, true)); + } +} From 81f9e1376cefed57022de62cd3abf5641c46e4aa Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 7 Mar 2017 01:29:56 +0000 Subject: [PATCH 075/101] MINOR: Rename RecordBatch to ProducerBatch to free the name for KIP-98 Author: Jason Gustafson Reviewers: Apurva Mehta , Ismael Juma Closes #2646 from hachikuji/rename-record-batch --- .../{RecordBatch.java => ProducerBatch.java} | 8 +- .../producer/internals/RecordAccumulator.java | 100 +++++++++--------- .../clients/producer/internals/Sender.java | 40 +++---- ...dBatchTest.java => ProducerBatchTest.java} | 25 ++--- .../internals/RecordAccumulatorTest.java | 36 +++---- 5 files changed, 105 insertions(+), 104 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/producer/internals/{RecordBatch.java => ProducerBatch.java} (96%) rename clients/src/test/java/org/apache/kafka/clients/producer/internals/{RecordBatchTest.java => ProducerBatchTest.java} (66%) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java similarity index 96% rename from clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java index 8dacaf5fe9687..46273a12d3004 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java @@ -36,9 +36,9 @@ * * This class is not thread safe and external synchronization must be used when modifying it */ -public final class RecordBatch { +public final class ProducerBatch { - private static final Logger log = LoggerFactory.getLogger(RecordBatch.class); + private static final Logger log = LoggerFactory.getLogger(ProducerBatch.class); final long createdMs; final TopicPartition topicPartition; @@ -57,7 +57,7 @@ public final class RecordBatch { private AtomicBoolean completed; private boolean retry; - public RecordBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long now) { + public ProducerBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long now) { this.createdMs = now; this.lastAttemptMs = now; this.recordsBuilder = recordsBuilder; @@ -139,7 +139,7 @@ public Thunk(Callback callback, FutureRecordMetadata future) { @Override public String toString() { - return "RecordBatch(topicPartition=" + topicPartition + ", recordCount=" + recordCount + ")"; + return "ProducerBatch(topicPartition=" + topicPartition + ", recordCount=" + recordCount + ")"; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 330682085f3fe..5d95f53a55863 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -73,8 +73,8 @@ public final class RecordAccumulator { private final long retryBackoffMs; private final BufferPool free; private final Time time; - private final ConcurrentMap> batches; - private final IncompleteRecordBatches incomplete; + private final ConcurrentMap> batches; + private final IncompleteBatches incomplete; // The following variables are only accessed by the sender thread, so we don't need to protect them. private final Set muted; private int drainIndex; @@ -111,7 +111,7 @@ public RecordAccumulator(int batchSize, this.batches = new CopyOnWriteMap<>(); String metricGrpName = "producer-metrics"; this.free = new BufferPool(totalSize, batchSize, metrics, time, metricGrpName); - this.incomplete = new IncompleteRecordBatches(); + this.incomplete = new IncompleteBatches(); this.muted = new HashSet<>(); this.time = time; registerMetrics(metrics, metricGrpName); @@ -172,7 +172,7 @@ public RecordAppendResult append(TopicPartition tp, ByteBuffer buffer = null; try { // check if we have an in-progress batch - Deque dq = getOrCreateDeque(tp); + Deque dq = getOrCreateDeque(tp); synchronized (dq) { if (closed) throw new IllegalStateException("Cannot send after the producer is closed."); @@ -197,7 +197,7 @@ public RecordAppendResult append(TopicPartition tp, } MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(buffer, compression, TimestampType.CREATE_TIME, this.batchSize); - RecordBatch batch = new RecordBatch(tp, recordsBuilder, time.milliseconds()); + ProducerBatch batch = new ProducerBatch(tp, recordsBuilder, time.milliseconds()); FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds())); dq.addLast(batch); @@ -216,11 +216,11 @@ public RecordAppendResult append(TopicPartition tp, } /** - * If `RecordBatch.tryAppend` fails (i.e. the record batch is full), close its memory records to release temporary + * If `ProducerBatch.tryAppend` fails (i.e. the record batch is full), close its memory records to release temporary * resources (like compression streams buffers). */ - private RecordAppendResult tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, Deque deque) { - RecordBatch last = deque.peekLast(); + private RecordAppendResult tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, Deque deque) { + ProducerBatch last = deque.peekLast(); if (last != null) { FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds()); if (future == null) @@ -235,11 +235,11 @@ private RecordAppendResult tryAppend(long timestamp, byte[] key, byte[] value, C * Abort the batches that have been sitting in RecordAccumulator for more than the configured requestTimeout * due to metadata being unavailable */ - public List abortExpiredBatches(int requestTimeout, long now) { - List expiredBatches = new ArrayList<>(); + public List abortExpiredBatches(int requestTimeout, long now) { + List expiredBatches = new ArrayList<>(); int count = 0; - for (Map.Entry> entry : this.batches.entrySet()) { - Deque dq = entry.getValue(); + for (Map.Entry> entry : this.batches.entrySet()) { + Deque dq = entry.getValue(); TopicPartition tp = entry.getKey(); // We only check if the batch should be expired if the partition does not have a batch in flight. // This is to prevent later batches from being expired while an earlier batch is still in progress. @@ -248,10 +248,10 @@ public List abortExpiredBatches(int requestTimeout, long now) { if (!muted.contains(tp)) { synchronized (dq) { // iterate over the batches and expire them if they have been in the accumulator for more than requestTimeOut - RecordBatch lastBatch = dq.peekLast(); - Iterator batchIterator = dq.iterator(); + ProducerBatch lastBatch = dq.peekLast(); + Iterator batchIterator = dq.iterator(); while (batchIterator.hasNext()) { - RecordBatch batch = batchIterator.next(); + ProducerBatch batch = batchIterator.next(); boolean isFull = batch != lastBatch || batch.isFull(); // Check if the batch has expired. Expired batches are closed by maybeExpire, but callbacks // are invoked after completing the iterations, since sends invoked from callbacks @@ -271,7 +271,7 @@ public List abortExpiredBatches(int requestTimeout, long now) { } if (!expiredBatches.isEmpty()) { log.trace("Expired {} batches in accumulator", count); - for (RecordBatch batch : expiredBatches) { + for (ProducerBatch batch : expiredBatches) { batch.expirationDone(); deallocate(batch); } @@ -283,9 +283,9 @@ public List abortExpiredBatches(int requestTimeout, long now) { /** * Re-enqueue the given record batch in the accumulator to retry */ - public void reenqueue(RecordBatch batch, long now) { + public void reenqueue(ProducerBatch batch, long now) { batch.reenqueued(now); - Deque deque = getOrCreateDeque(batch.topicPartition); + Deque deque = getOrCreateDeque(batch.topicPartition); synchronized (deque) { deque.addFirst(batch); } @@ -318,9 +318,9 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { Set unknownLeaderTopics = new HashSet<>(); boolean exhausted = this.free.queued() > 0; - for (Map.Entry> entry : this.batches.entrySet()) { + for (Map.Entry> entry : this.batches.entrySet()) { TopicPartition part = entry.getKey(); - Deque deque = entry.getValue(); + Deque deque = entry.getValue(); Node leader = cluster.leaderFor(part); synchronized (deque) { @@ -329,7 +329,7 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { // Note that entries are currently not removed from batches when deque is empty. unknownLeaderTopics.add(part.topic()); } else if (!readyNodes.contains(leader) && !muted.contains(part)) { - RecordBatch batch = deque.peekFirst(); + ProducerBatch batch = deque.peekFirst(); if (batch != null) { long waitedTimeMs = batch.waitedTimeMs(nowMs); boolean backingOff = batch.attempts() > 0 && waitedTimeMs < retryBackoffMs; @@ -358,8 +358,8 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { * @return Whether there is any unsent record in the accumulator. */ public boolean hasUnsent() { - for (Map.Entry> entry : this.batches.entrySet()) { - Deque deque = entry.getValue(); + for (Map.Entry> entry : this.batches.entrySet()) { + Deque deque = entry.getValue(); synchronized (deque) { if (!deque.isEmpty()) return true; @@ -376,20 +376,20 @@ public boolean hasUnsent() { * @param nodes The list of node to drain * @param maxSize The maximum number of bytes to drain * @param now The current unix time in milliseconds - * @return A list of {@link RecordBatch} for each node specified with total size less than the requested maxSize. + * @return A list of {@link ProducerBatch} for each node specified with total size less than the requested maxSize. */ - public Map> drain(Cluster cluster, - Set nodes, - int maxSize, - long now) { + public Map> drain(Cluster cluster, + Set nodes, + int maxSize, + long now) { if (nodes.isEmpty()) return Collections.emptyMap(); - Map> batches = new HashMap<>(); + Map> batches = new HashMap<>(); for (Node node : nodes) { int size = 0; List parts = cluster.partitionsForNode(node.id()); - List ready = new ArrayList<>(); + List ready = new ArrayList<>(); /* to make starvation less likely this loop doesn't start at 0 */ int start = drainIndex = drainIndex % parts.size(); do { @@ -397,10 +397,10 @@ public Map> drain(Cluster cluster, TopicPartition tp = new TopicPartition(part.topic(), part.partition()); // Only proceed if the partition has no in-flight batches. if (!muted.contains(tp)) { - Deque deque = getDeque(new TopicPartition(part.topic(), part.partition())); + Deque deque = getDeque(new TopicPartition(part.topic(), part.partition())); if (deque != null) { synchronized (deque) { - RecordBatch first = deque.peekFirst(); + ProducerBatch first = deque.peekFirst(); if (first != null) { boolean backoff = first.attempts() > 0 && first.waitedTimeMs(now) < retryBackoffMs; // Only drain the batch if it is not during backoff period. @@ -411,7 +411,7 @@ public Map> drain(Cluster cluster, // request break; } else { - RecordBatch batch = deque.pollFirst(); + ProducerBatch batch = deque.pollFirst(); batch.close(); size += batch.sizeInBytes(); ready.add(batch); @@ -429,19 +429,19 @@ public Map> drain(Cluster cluster, return batches; } - private Deque getDeque(TopicPartition tp) { + private Deque getDeque(TopicPartition tp) { return batches.get(tp); } /** * Get the deque for the given topic-partition, creating it if necessary. */ - private Deque getOrCreateDeque(TopicPartition tp) { - Deque d = this.batches.get(tp); + private Deque getOrCreateDeque(TopicPartition tp) { + Deque d = this.batches.get(tp); if (d != null) return d; d = new ArrayDeque<>(); - Deque previous = this.batches.putIfAbsent(tp, d); + Deque previous = this.batches.putIfAbsent(tp, d); if (previous == null) return d; else @@ -451,7 +451,7 @@ private Deque getOrCreateDeque(TopicPartition tp) { /** * Deallocate the record batch */ - public void deallocate(RecordBatch batch) { + public void deallocate(ProducerBatch batch) { incomplete.remove(batch); free.deallocate(batch.buffer(), batch.initialCapacity()); } @@ -466,7 +466,7 @@ boolean flushInProgress() { } /* Visible for testing */ - Map> batches() { + Map> batches() { return Collections.unmodifiableMap(batches); } @@ -489,7 +489,7 @@ private boolean appendsInProgress() { */ public void awaitFlushCompletion() throws InterruptedException { try { - for (RecordBatch batch : this.incomplete.all()) + for (ProducerBatch batch : this.incomplete.all()) batch.produceFuture.await(); } finally { this.flushesInProgress.decrementAndGet(); @@ -519,8 +519,8 @@ public void abortIncompleteBatches() { * Go through incomplete batches and abort them. */ private void abortBatches() { - for (RecordBatch batch : incomplete.all()) { - Deque dq = getDeque(batch.topicPartition); + for (ProducerBatch batch : incomplete.all()) { + Deque dq = getDeque(batch.topicPartition); // Close the batch before aborting synchronized (dq) { batch.close(); @@ -577,22 +577,22 @@ public ReadyCheckResult(Set readyNodes, long nextReadyCheckDelayMs, Set incomplete; + private final static class IncompleteBatches { + private final Set incomplete; - public IncompleteRecordBatches() { - this.incomplete = new HashSet(); + public IncompleteBatches() { + this.incomplete = new HashSet<>(); } - public void add(RecordBatch batch) { + public void add(ProducerBatch batch) { synchronized (incomplete) { this.incomplete.add(batch); } } - public void remove(RecordBatch batch) { + public void remove(ProducerBatch batch) { synchronized (incomplete) { boolean removed = this.incomplete.remove(batch); if (!removed) @@ -600,7 +600,7 @@ public void remove(RecordBatch batch) { } } - public Iterable all() { + public Iterable all() { synchronized (incomplete) { return new ArrayList<>(this.incomplete); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 7f27d36845acb..3604f68fd5c90 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -192,21 +192,21 @@ void run(long now) { } // create produce requests - Map> batches = this.accumulator.drain(cluster, + Map> batches = this.accumulator.drain(cluster, result.readyNodes, this.maxRequestSize, now); if (guaranteeMessageOrder) { // Mute all the partitions drained - for (List batchList : batches.values()) { - for (RecordBatch batch : batchList) + for (List batchList : batches.values()) { + for (ProducerBatch batch : batchList) this.accumulator.mutePartition(batch.topicPartition); } } - List expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, now); + List expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, now); // update sensors - for (RecordBatch expiredBatch : expiredBatches) + for (ProducerBatch expiredBatch : expiredBatches) this.sensors.recordErrors(expiredBatch.topicPartition.topic(), expiredBatch.recordCount); sensors.updateProduceRequestMetrics(batches); @@ -251,16 +251,16 @@ public void forceClose() { /** * Handle a produce response */ - private void handleProduceResponse(ClientResponse response, Map batches, long now) { + private void handleProduceResponse(ClientResponse response, Map batches, long now) { int correlationId = response.requestHeader().correlationId(); if (response.wasDisconnected()) { log.trace("Cancelled request {} due to node {} being disconnected", response, response.destination()); - for (RecordBatch batch : batches.values()) + for (ProducerBatch batch : batches.values()) completeBatch(batch, new ProduceResponse.PartitionResponse(Errors.NETWORK_EXCEPTION), correlationId, now); } else if (response.versionMismatch() != null) { log.warn("Cancelled request {} due to a version mismatch with node {}", response, response.destination(), response.versionMismatch()); - for (RecordBatch batch : batches.values()) + for (ProducerBatch batch : batches.values()) completeBatch(batch, new ProduceResponse.PartitionResponse(Errors.INVALID_REQUEST), correlationId, now); } else { log.trace("Received produce response from node {} with correlation id {}", response.destination(), correlationId); @@ -270,14 +270,14 @@ private void handleProduceResponse(ClientResponse response, Map entry : produceResponse.responses().entrySet()) { TopicPartition tp = entry.getKey(); ProduceResponse.PartitionResponse partResp = entry.getValue(); - RecordBatch batch = batches.get(tp); + ProducerBatch batch = batches.get(tp); completeBatch(batch, partResp, correlationId, now); } this.sensors.recordLatency(response.destination(), response.requestLatencyMs()); this.sensors.recordThrottleTime(produceResponse.getThrottleTime()); } else { // this is the acks = 0 case, just complete all requests - for (RecordBatch batch : batches.values()) { + for (ProducerBatch batch : batches.values()) { completeBatch(batch, new ProduceResponse.PartitionResponse(Errors.NONE), correlationId, now); } } @@ -292,7 +292,7 @@ private void handleProduceResponse(ClientResponse response, Map> collated, long now) { - for (Map.Entry> entry : collated.entrySet()) + private void sendProduceRequests(Map> collated, long now) { + for (Map.Entry> entry : collated.entrySet()) sendProduceRequest(now, entry.getKey(), acks, requestTimeout, entry.getValue()); } /** * Create a produce request from the given record batches */ - private void sendProduceRequest(long now, int destination, short acks, int timeout, List batches) { + private void sendProduceRequest(long now, int destination, short acks, int timeout, List batches) { Map produceRecordsByPartition = new HashMap<>(batches.size()); - final Map recordsByPartition = new HashMap<>(batches.size()); - for (RecordBatch batch : batches) { + final Map recordsByPartition = new HashMap<>(batches.size()); + for (ProducerBatch batch : batches) { TopicPartition tp = batch.topicPartition; produceRecordsByPartition.put(tp, batch.records()); recordsByPartition.put(tp, batch); @@ -493,11 +493,11 @@ private void maybeRegisterTopicMetrics(String topic) { } } - public void updateProduceRequestMetrics(Map> batches) { + public void updateProduceRequestMetrics(Map> batches) { long now = time.milliseconds(); - for (List nodeBatch : batches.values()) { + for (List nodeBatch : batches.values()) { int records = 0; - for (RecordBatch batch : nodeBatch) { + for (ProducerBatch batch : nodeBatch) { // register all per-topic metrics at once String topic = batch.topicPartition.topic(); maybeRegisterTopicMetrics(topic); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordBatchTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java similarity index 66% rename from clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordBatchTest.java rename to clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java index 6404451b4dff9..3258ba3ed67b8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordBatchTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.producer.internals; -import java.nio.ByteBuffer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecordsBuilder; @@ -24,9 +23,11 @@ import org.apache.kafka.common.record.TimestampType; import org.junit.Test; +import java.nio.ByteBuffer; + import static org.junit.Assert.assertFalse; -public class RecordBatchTest { +public class ProducerBatchTest { private final long now = 1488748346917L; @@ -34,25 +35,25 @@ public class RecordBatchTest { Record.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, 0L, Record.NO_TIMESTAMP, 0); /** - * A RecordBatch configured using a very large linger value and a timestamp preceding its create + * A {@link ProducerBatch} configured using a very large linger value and a timestamp preceding its create * time is interpreted correctly as not expired when the linger time is larger than the difference - * between now and create time by RecordBatch#maybeExpire. + * between now and create time by {@link ProducerBatch#maybeExpire(int, long, long, long, boolean)}. */ @Test public void testLargeLingerOldNowExpire() { - RecordBatch batch = new RecordBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); + ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); // Set `now` to 2ms before the create time. assertFalse(batch.maybeExpire(10240, 100L, now - 2L, Long.MAX_VALUE, false)); } /** - * A RecordBatch configured using a very large retryBackoff value with retry = true and a timestamp preceding its - * create time is interpreted correctly as not expired when the retryBackoff time is larger than the difference - * between now and create time by RecordBatch#maybeExpire. + * A {@link ProducerBatch} configured using a very large retryBackoff value with retry = true and a timestamp + * preceding its create time is interpreted correctly as not expired when the retryBackoff time is larger than the + * difference between now and create time by {@link ProducerBatch#maybeExpire(int, long, long, long, boolean)}. */ @Test public void testLargeRetryBackoffOldNowExpire() { - RecordBatch batch = new RecordBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); + ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); // Set batch.retry = true batch.reenqueued(now); // Set `now` to 2ms before the create time. @@ -60,12 +61,12 @@ public void testLargeRetryBackoffOldNowExpire() { } /** - * A RecordBatch#maybeExpire call with a now value before the create time of the RecordBatch is correctly recognized - * as not expired when invoked with parameter isFull = true. + * A {@link ProducerBatch#maybeExpire(int, long, long, long, boolean)} call with a now value before the create + * time of the ProducerBatch is correctly recognized as not expired when invoked with parameter isFull = true. */ @Test public void testLargeFullOldNowExpire() { - RecordBatch batch = new RecordBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); + ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); // Set `now` to 2ms before the create time. assertFalse(batch.maybeExpire(10240, 10240L, now - 2L, 10240L, true)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 1cb510e3f2555..42dc4c4a34025 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -89,7 +89,7 @@ public void testFull() throws Exception { for (int i = 0; i < appends; i++) { // append to the first batch accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); - Deque partitionBatches = accum.batches().get(tp1); + Deque partitionBatches = accum.batches().get(tp1); assertEquals(1, partitionBatches.size()); assertTrue(partitionBatches.peekFirst().isWritable()); assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); @@ -97,16 +97,16 @@ public void testFull() throws Exception { // this append doesn't fit in the first batch, so a new batch is created and the first batch is closed accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); - Deque partitionBatches = accum.batches().get(tp1); + Deque partitionBatches = accum.batches().get(tp1); assertEquals(2, partitionBatches.size()); - Iterator partitionBatchesIterator = partitionBatches.iterator(); + Iterator partitionBatchesIterator = partitionBatches.iterator(); assertFalse(partitionBatchesIterator.next().isWritable()); assertTrue(partitionBatchesIterator.next().isWritable()); assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); + List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); assertEquals(1, batches.size()); - RecordBatch batch = batches.get(0); + ProducerBatch batch = batches.get(0); Iterator iter = batch.records().deepEntries().iterator(); for (int i = 0; i < appends; i++) { @@ -133,9 +133,9 @@ public void testLinger() throws Exception { assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); time.sleep(10); assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); + List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); assertEquals(1, batches.size()); - RecordBatch batch = batches.get(0); + ProducerBatch batch = batches.get(0); Iterator iter = batch.records().deepEntries().iterator(); LogEntry entry = iter.next(); @@ -155,7 +155,7 @@ public void testPartialDrain() throws Exception { } assertEquals("Partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node1), 1024, 0).get(node1.id()); + List batches = accum.drain(cluster, Collections.singleton(node1), 1024, 0).get(node1.id()); assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size()); } @@ -186,9 +186,9 @@ public void run() { long now = time.milliseconds(); while (read < numThreads * msgs) { Set nodes = accum.ready(cluster, now).readyNodes; - List batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id()); + List batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id()); if (batches != null) { - for (RecordBatch batch : batches) { + for (ProducerBatch batch : batches) { for (LogEntry entry : batch.records().deepEntries()) read++; accum.deallocate(batch); @@ -245,7 +245,7 @@ public void testRetryBackoff() throws Exception { accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, now + lingerMs + 1); assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes); - Map> batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1); + Map> batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1); assertEquals("Node1 should be the only ready node.", 1, batches.size()); assertEquals("Partition 0 should only have one batch drained.", 1, batches.get(0).size()); @@ -286,9 +286,9 @@ public void testFlush() throws Exception { result = accum.ready(cluster, time.milliseconds()); // drain and deallocate all batches - Map> results = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); - for (List batches: results.values()) - for (RecordBatch batch: batches) + Map> results = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); + for (List batches: results.values()) + for (ProducerBatch batch: batches) accum.deallocate(batch); // should be complete with no unsent records. @@ -368,7 +368,7 @@ public void testExpiredBatches() throws InterruptedException { // Advance the clock to expire the batch. time.sleep(requestTimeout + 1); accum.mutePartition(tp1); - List expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); + List expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should not be expired when the partition is muted", 0, expiredBatches.size()); accum.unmutePartition(tp1); @@ -396,7 +396,7 @@ public void testExpiredBatches() throws InterruptedException { time.sleep(lingerMs); readyNodes = accum.ready(cluster, time.milliseconds()).readyNodes; assertEquals("Our partition's leader should be ready", Collections.singleton(node1), readyNodes); - Map> drained = accum.drain(cluster, readyNodes, Integer.MAX_VALUE, time.milliseconds()); + Map> drained = accum.drain(cluster, readyNodes, Integer.MAX_VALUE, time.milliseconds()); assertEquals("There should be only one batch.", drained.get(node1.id()).size(), 1); time.sleep(1000L); accum.reenqueue(drained.get(node1.id()).get(0), time.milliseconds()); @@ -450,7 +450,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { // Advance the clock to expire the first batch. time.sleep(requestTimeout + 1); - List expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); + List expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch was not expired", 1, expiredBatches.size()); assertEquals("Callbacks not invoked for expiry", messagesPerBatch, expiryCallbackCount.get()); assertNull("Unexpected exception", unexpectedException.get()); @@ -481,7 +481,7 @@ public void testMutedPartitions() throws InterruptedException { // Test drain with muted partition accum.mutePartition(tp1); - Map> drained = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); + Map> drained = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); assertEquals("No batch should have been drained", 0, drained.get(node1.id()).size()); // Test drain without muted partition. From 146edd530a6dfa534fd8cb924fd2336e27872c2a Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Tue, 7 Mar 2017 15:31:22 -0800 Subject: [PATCH 076/101] KAFKA-4851: only search available segments during Segments.segments(from, to) restrict the locating of segments in `Segments#segments(..)` to only the segments that are currently available, i.e., rather than searching the hashmap for many segments that don't exist. Author: Damian Guy Reviewers: Guozhang Wang Closes #2645 from dguy/session-windows-testing --- .../streams/state/internals/Segments.java | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java index 41cef54b50777..5dedb409205ea 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java @@ -41,8 +41,8 @@ class Segments { private final int numSegments; private final long segmentInterval; private final SimpleDateFormat formatter; - - private long currentSegmentId = -1L; + private long minSegmentId = Long.MAX_VALUE; + private long maxSegmentId = -1L; Segments(final String name, final long retentionPeriod, final int numSegments) { this.name = name; @@ -66,7 +66,7 @@ Segment getSegmentForTimestamp(final long timestamp) { } Segment getOrCreateSegment(final long segmentId, final ProcessorContext context) { - if (segmentId > currentSegmentId || segmentId > currentSegmentId - numSegments) { + if (segmentId > maxSegmentId - numSegments) { final long key = segmentId % numSegments; final Segment segment = segments.get(key); if (!isSegment(segment, segmentId)) { @@ -76,7 +76,10 @@ Segment getOrCreateSegment(final long segmentId, final ProcessorContext context) Segment newSegment = new Segment(segmentName(segmentId), name, segmentId); newSegment.openDB(context); segments.put(key, newSegment); - currentSegmentId = segmentId > currentSegmentId ? segmentId : currentSegmentId; + maxSegmentId = segmentId > maxSegmentId ? segmentId : maxSegmentId; + if (minSegmentId == Long.MAX_VALUE) { + minSegmentId = maxSegmentId; + } } return segments.get(key); } else { @@ -113,8 +116,8 @@ void openExisting(final ProcessorContext context) { } List segments(final long timeFrom, final long timeTo) { - final long segFrom = segmentId(Math.max(0L, timeFrom)); - final long segTo = segmentId(Math.min(currentSegmentId * segmentInterval, Math.max(0, timeTo))); + final long segFrom = Math.max(minSegmentId, segmentId(Math.max(0L, timeFrom))); + final long segTo = Math.min(maxSegmentId, segmentId(Math.min(maxSegmentId * segmentInterval, Math.max(0, timeTo)))); final List segments = new ArrayList<>(); for (long segmentId = segFrom; segmentId <= segTo; segmentId++) { @@ -155,9 +158,9 @@ private boolean isSegment(final Segment store, long segmentId) { } private void cleanup(final long segmentId) { - final long oldestSegmentId = currentSegmentId < segmentId + final long oldestSegmentId = maxSegmentId < segmentId ? segmentId - numSegments - : currentSegmentId - numSegments; + : maxSegmentId - numSegments; for (Map.Entry segmentEntry : segments.entrySet()) { final Segment segment = segmentEntry.getValue(); @@ -167,6 +170,9 @@ private void cleanup(final long segmentId) { segment.destroy(); } } + if (oldestSegmentId > minSegmentId) { + minSegmentId = oldestSegmentId + 1; + } } private long segmentIdFromSegmentName(String segmentName) { From c6bccddb94affe8f85110004341f75bd66b839eb Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Wed, 8 Mar 2017 00:44:28 +0000 Subject: [PATCH 077/101] KAFKA-4841; NetworkClient should only consider a connection to have failed after attempt to connect Also fix a potential reordering bug and include a few clean-ups. Author: Dong Lin Reviewers: Jiangjie (Becket) Qin , Jason Gustafson , Ismael Juma Closes #2641 from lindong28/KAFKA-4820-followup --- .../clients/ClusterConnectionStates.java | 13 ++- .../apache/kafka/clients/NetworkClient.java | 2 +- .../internals/ConsumerNetworkClient.java | 95 ++++++++----------- 3 files changed, 52 insertions(+), 58 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java index 0ce799370e11a..9bde1a22460f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -36,7 +36,7 @@ public ClusterConnectionStates(long reconnectBackoffMs) { * Return true iff we can currently initiate a new connection. This will be the case if we are not * connected and haven't been connected for at least the minimum reconnection backoff period. * @param id the connection id to check - * @param now the current time in MS + * @param now the current time in ms * @return true if we can initiate a new connection */ public boolean canConnect(String id, long now) { @@ -136,6 +136,15 @@ public boolean isReady(String id) { return state != null && state.state == ConnectionState.READY; } + /** + * Return true if the connection has been disconnected + * @param id The id of the node to check + */ + public boolean isDisconnected(String id) { + NodeConnectionState state = nodeState.get(id); + return state != null && state.state == ConnectionState.DISCONNECTED; + } + /** * Remove the given node from the tracked connection states. The main difference between this and `disconnected` * is the impact on `connectionDelay`: it will be 0 after this call whereas `reconnectBackoffMs` will be taken @@ -155,7 +164,7 @@ public void remove(String id) { public ConnectionState connectionState(String id) { return nodeState(id).state; } - + /** * Get the state of a given node. * @param id the connection to fetch the state for diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 8d3adda0b386a..c76a738f071c6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -229,7 +229,7 @@ public long connectionDelay(Node node, long now) { */ @Override public boolean connectionFailed(Node node) { - return connectionStates.connectionState(node.idString()).equals(ConnectionState.DISCONNECTED); + return connectionStates.isDisconnected(node.idString()); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 2fa766782e2b7..eb25359a05d42 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -35,13 +35,12 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.kafka.common.errors.InterruptException; @@ -288,7 +287,7 @@ public boolean awaitPendingRequests(Node node, long timeoutMs) { */ public int pendingRequestCount(Node node) { synchronized (this) { - return unsent.getRequestCount(node) + client.inFlightRequestCount(node.idString()); + return unsent.requestCount(node) + client.inFlightRequestCount(node.idString()); } } @@ -313,7 +312,7 @@ public boolean hasPendingRequest(Node node) { */ public int pendingRequestCount() { synchronized (this) { - return unsent.getRequestCount() + client.inFlightRequestCount(); + return unsent.requestCount() + client.inFlightRequestCount(); } } @@ -351,12 +350,12 @@ private void checkDisconnects(long now) { // by NetworkClient, so we just need to check whether connections for any of the unsent // requests have been disconnected; if they have, then we complete the corresponding future // and set the disconnect flag in the ClientResponse - Set nodes = unsent.getNodes(); - for (Node node: nodes) { + List nodes = unsent.nodes(); + for (Node node : nodes) { if (client.connectionFailed(node)) { // Remove entry before invoking request callback to avoid callbacks handling // coordinator failures traversing the unsent list again. - List requests = unsent.remove(node); + Collection requests = unsent.remove(node); for (ClientRequest request : requests) { RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); handler.onComplete(new ClientResponse(request.makeHeader(request.requestBuilder().desiredOrLatestVersion()), @@ -370,7 +369,7 @@ private void checkDisconnects(long now) { private void failExpiredRequests(long now) { // clear all expired unsent requests and fail their corresponding futures List expiredRequests = unsent.removeExpiredRequests(now, unsentExpiryMs); - for (ClientRequest request: expiredRequests) { + for (ClientRequest request : expiredRequests) { RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); handler.onFailure(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); } @@ -379,7 +378,7 @@ private void failExpiredRequests(long now) { public void failUnsentRequests(Node node, RuntimeException e) { // clear unsent requests to node and fail their corresponding futures synchronized (this) { - List unsentRequests = unsent.remove(node); + Collection unsentRequests = unsent.remove(node); for (ClientRequest unsentRequest : unsentRequests) { RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) unsentRequest.callback(); handler.onFailure(e); @@ -393,25 +392,15 @@ public void failUnsentRequests(Node node, RuntimeException e) { private boolean trySend(long now) { // send any requests that can be sent now boolean requestsSent = false; - Set nodes = unsent.getNodes(); - for (Node node: nodes) { - if (client.ready(node, now)) { - // Remove entry before invoking request callback to avoid callbacks handling - // coordinator failures traversing the unsent list again. - List requests = unsent.remove(node); - try { - Iterator iterator = requests.iterator(); - while (iterator.hasNext()) { - ClientRequest request = iterator.next(); - if (!client.ready(node, now)) - break; - client.send(request, now); - requestsSent = true; - iterator.remove(); - } - } finally { - if (!requests.isEmpty()) - unsent.put(node, requests); + + for (Node node : unsent.nodes()) { + Iterator iterator = unsent.requestIterator(node); + while (iterator.hasNext()) { + ClientRequest request = iterator.next(); + if (client.ready(node, now)) { + client.send(request, now); + iterator.remove(); + requestsSent = true; } } } @@ -538,52 +527,43 @@ public interface PollCondition { /* - * A threadsafe helper class to hold requests per node that has not been sent yet + * A threadsafe helper class to hold requests per node that have not been sent yet */ private final static class UnsentRequests { - private final Map> unsent; + private final Map> unsent; public UnsentRequests() { unsent = new HashMap<>(); } - public synchronized void put(Node node, List requests) { - List nodeUnsent = unsent.get(node); - if (nodeUnsent == null) { - nodeUnsent = new ArrayList<>(); - unsent.put(node, nodeUnsent); - } - nodeUnsent.addAll(requests); - } - public synchronized void put(Node node, ClientRequest request) { - List nodeUnsent = unsent.get(node); + ConcurrentLinkedQueue nodeUnsent = unsent.get(node); if (nodeUnsent == null) { - nodeUnsent = new ArrayList<>(); + nodeUnsent = new ConcurrentLinkedQueue<>(); unsent.put(node, nodeUnsent); } nodeUnsent.add(request); } - public synchronized int getRequestCount(Node node) { - List requests = unsent.get(node); + public synchronized int requestCount(Node node) { + ConcurrentLinkedQueue requests = unsent.get(node); return requests == null ? 0 : requests.size(); } - public synchronized int getRequestCount() { + public synchronized int requestCount() { int total = 0; - for (List requests : unsent.values()) + for (ConcurrentLinkedQueue requests : unsent.values()) total += requests.size(); return total; } public synchronized boolean hasRequest(Node node) { - List requests = unsent.get(node); + ConcurrentLinkedQueue requests = unsent.get(node); return requests != null && !requests.isEmpty(); } public synchronized boolean hasRequest() { - for (List requests : unsent.values()) + for (ConcurrentLinkedQueue requests : unsent.values()) if (!requests.isEmpty()) return true; return false; @@ -591,10 +571,10 @@ public synchronized boolean hasRequest() { public synchronized List removeExpiredRequests(long now, long unsentExpiryMs) { List expiredRequests = new ArrayList<>(); - Iterator>> iterator = unsent.entrySet().iterator(); + Iterator> iterator = unsent.values().iterator(); while (iterator.hasNext()) { - Map.Entry> requestEntry = iterator.next(); - Iterator requestIterator = requestEntry.getValue().iterator(); + ConcurrentLinkedQueue requests = iterator.next(); + Iterator requestIterator = requests.iterator(); while (requestIterator.hasNext()) { ClientRequest request = requestIterator.next(); if (request.createdTimeMs() < now - unsentExpiryMs) { @@ -603,19 +583,24 @@ public synchronized List removeExpiredRequests(long now, long uns } else break; } - if (requestEntry.getValue().isEmpty()) + if (requests.isEmpty()) iterator.remove(); } return expiredRequests; } - public synchronized List remove(Node node) { - List requests = unsent.remove(node); + public synchronized Collection remove(Node node) { + ConcurrentLinkedQueue requests = unsent.remove(node); return requests == null ? Collections.emptyList() : requests; } - public synchronized Set getNodes() { - return new HashSet<>(unsent.keySet()); + public synchronized Iterator requestIterator(Node node) { + ConcurrentLinkedQueue requests = unsent.get(node); + return requests == null ? Collections.emptyIterator() : requests.iterator(); + } + + public synchronized List nodes() { + return new ArrayList<>(unsent.keySet()); } } From 29084a9b27b959532813141591ce03be2921e860 Mon Sep 17 00:00:00 2001 From: "sharad.develop" Date: Wed, 8 Mar 2017 11:40:40 -0800 Subject: [PATCH 078/101] KAFKA-4722: Add application.id to StreamThread name Add application.id to StreamThread name Author: sharad.develop Reviewers: Matthias J. Sax, Guozhang Wang Closes #2617 from sharad-develop/KAFKA-4722 --- .../kafka/streams/processor/internals/StreamThread.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index d447824492253..9a2c3fac1b116 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -236,9 +236,8 @@ public StreamThread(TopologyBuilder builder, Time time, StreamsMetadataState streamsMetadataState, final long cacheSizeBytes) { - super("StreamThread-" + STREAM_THREAD_ID_SEQUENCE.getAndIncrement()); + super(clientId + "-StreamThread-" + STREAM_THREAD_ID_SEQUENCE.getAndIncrement()); this.applicationId = applicationId; - String threadName = getName(); this.config = config; this.builder = builder; this.sourceTopicPattern = builder.sourceTopicPattern(); @@ -246,16 +245,16 @@ public StreamThread(TopologyBuilder builder, this.processId = processId; this.partitionGrouper = config.getConfiguredInstance(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, PartitionGrouper.class); this.streamsMetadataState = streamsMetadataState; - threadClientId = clientId + "-" + threadName; + threadClientId = getName(); this.streamsMetrics = new StreamsMetricsThreadImpl(metrics, "stream-metrics", "thread." + threadClientId, Collections.singletonMap("client-id", threadClientId)); if (config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) < 0) { - log.warn("Negative cache size passed in thread [{}]. Reverting to cache size of 0 bytes.", threadName); + log.warn("Negative cache size passed in thread [{}]. Reverting to cache size of 0 bytes.", threadClientId); } this.cache = new ThreadCache(threadClientId, cacheSizeBytes, this.streamsMetrics); - this.logPrefix = String.format("stream-thread [%s]", threadName); + this.logPrefix = String.format("stream-thread [%s]", threadClientId); // set the producer and consumer clients log.info("{} Creating producer client", logPrefix); From f7354e779cb058985dd385b2b1209db71eb7a5e8 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 8 Mar 2017 22:25:58 +0000 Subject: [PATCH 079/101] MINOR: Add varint serde utilities for new message format Author: Jason Gustafson Reviewers: Ismael Juma Closes #2647 from hachikuji/add-varint-serdes --- .../kafka/common/protocol/types/Type.java | 57 +++ .../record/ByteBufferLogInputStream.java | 4 +- .../kafka/common/record/CompressionType.java | 2 + .../record/KafkaLZ4BlockInputStream.java | 10 +- .../record/KafkaLZ4BlockOutputStream.java | 10 +- .../common/record/MemoryRecordsBuilder.java | 1 + .../apache/kafka/common/record/Record.java | 6 +- .../kafka/common/record/RecordsIterator.java | 1 + .../ByteBufferInputStream.java | 2 +- .../ByteBufferOutputStream.java | 2 +- .../apache/kafka/common/utils/ByteUtils.java | 324 ++++++++++++++++++ .../org/apache/kafka/common/utils/Utils.java | 105 +----- .../consumer/internals/FetcherTest.java | 2 +- .../types/ProtocolSerializationTest.java | 8 + .../common/record/CompressionTypeTest.java | 2 + .../kafka/common/record/SimpleRecordTest.java | 1 + .../kafka/common/utils/ByteUtilsTest.java | 222 ++++++++++++ .../apache/kafka/common/utils/UtilsTest.java | 80 +---- .../main/scala/kafka/message/Message.scala | 7 +- .../unit/kafka/message/MessageTest.scala | 5 +- .../server/AbstractFetcherThreadTest.scala | 4 +- .../internals/assignment/AssignmentInfo.java | 2 +- 22 files changed, 668 insertions(+), 189 deletions(-) rename clients/src/main/java/org/apache/kafka/common/{record => utils}/ByteBufferInputStream.java (97%) rename clients/src/main/java/org/apache/kafka/common/{record => utils}/ByteBufferOutputStream.java (97%) create mode 100644 clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java create mode 100644 clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java index 3341f3ef69329..39e46fd97178f 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.ByteUtils; import org.apache.kafka.common.utils.Utils; /** @@ -473,4 +474,60 @@ public Records validate(Object item) { } }; + public static final Type VARINT = new Type() { + @Override + public void write(ByteBuffer buffer, Object o) { + ByteUtils.writeVarint((Integer) o, buffer); + } + + @Override + public Integer read(ByteBuffer buffer) { + return ByteUtils.readVarint(buffer); + } + + @Override + public Integer validate(Object item) { + if (item instanceof Integer) + return (Integer) item; + throw new SchemaException(item + " is not an integer"); + } + + public String toString() { + return "VARINT"; + } + + @Override + public int sizeOf(Object o) { + return ByteUtils.sizeOfVarint((Integer) o); + } + }; + + public static final Type VARLONG = new Type() { + @Override + public void write(ByteBuffer buffer, Object o) { + ByteUtils.writeVarlong((Long) o, buffer); + } + + @Override + public Long read(ByteBuffer buffer) { + return ByteUtils.readVarlong(buffer); + } + + @Override + public Long validate(Object item) { + if (item instanceof Long) + return (Long) item; + throw new SchemaException(item + " is not a long"); + } + + public String toString() { + return "VARLONG"; + } + + @Override + public int sizeOf(Object o) { + return ByteUtils.sizeOfVarlong((Long) o); + } + }; + } diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java index bdda998909b6c..f4a3da47f28ae 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java @@ -17,7 +17,7 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.errors.CorruptRecordException; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.utils.ByteUtils; import java.io.IOException; import java.nio.ByteBuffer; @@ -106,7 +106,7 @@ private void setTimestampAndUpdateCrc(TimestampType timestampType, long timestam buffer.put(LOG_OVERHEAD + Record.ATTRIBUTES_OFFSET, timestampType.updateAttributes(attributes)); buffer.putLong(LOG_OVERHEAD + Record.TIMESTAMP_OFFSET, timestamp); long crc = record.computeChecksum(); - Utils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc); + ByteUtils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc); } public ByteBuffer buffer() { diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java index 658e50ca6a8cf..d88c530bc8ce0 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java @@ -17,6 +17,8 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.ByteBufferInputStream; +import org.apache.kafka.common.utils.ByteBufferOutputStream; import java.io.InputStream; import java.io.OutputStream; diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java index 6544d13b1347d..a53690c72aff7 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java @@ -26,7 +26,7 @@ import org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.BD; import org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.FLG; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.utils.ByteUtils; import net.jpountz.lz4.LZ4Exception; import net.jpountz.lz4.LZ4Factory; @@ -112,7 +112,7 @@ private void readHeader() throws IOException { throw new IOException(PREMATURE_EOS); } - if (MAGIC != Utils.readUnsignedIntLE(header, headerOffset - 6)) { + if (MAGIC != ByteUtils.readUnsignedIntLE(header, headerOffset - 6)) { throw new IOException(NOT_SUPPORTED); } flg = FLG.fromByte(header[headerOffset - 2]); @@ -145,13 +145,13 @@ private void readHeader() throws IOException { * @throws IOException */ private void readBlock() throws IOException { - int blockSize = Utils.readUnsignedIntLE(in); + int blockSize = ByteUtils.readUnsignedIntLE(in); // Check for EndMark if (blockSize == 0) { finished = true; if (flg.isContentChecksumSet()) - Utils.readUnsignedIntLE(in); // TODO: verify this content checksum + ByteUtils.readUnsignedIntLE(in); // TODO: verify this content checksum return; } else if (blockSize > maxBlockSize) { throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize)); @@ -172,7 +172,7 @@ private void readBlock() throws IOException { } // verify checksum - if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) { + if (flg.isBlockChecksumSet() && ByteUtils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) { throw new IOException(BLOCK_HASH_MISMATCH); } diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java index 0a64d43e6344f..034b94521f60b 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.io.OutputStream; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.utils.ByteUtils; import net.jpountz.lz4.LZ4Compressor; import net.jpountz.lz4.LZ4Factory; @@ -138,7 +138,7 @@ public boolean useBrokenFlagDescriptorChecksum() { * @throws IOException */ private void writeHeader() throws IOException { - Utils.writeUnsignedIntLE(buffer, 0, MAGIC); + ByteUtils.writeUnsignedIntLE(buffer, 0, MAGIC); bufferOffset = 4; buffer[bufferOffset++] = flg.toByte(); buffer[bufferOffset++] = bd.toByte(); @@ -182,13 +182,13 @@ private void writeBlock() throws IOException { } // Write content - Utils.writeUnsignedIntLE(out, compressedLength | compressMethod); + ByteUtils.writeUnsignedIntLE(out, compressedLength | compressMethod); out.write(bufferToWrite, 0, compressedLength); // Calculate and write block checksum if (flg.isBlockChecksumSet()) { int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0); - Utils.writeUnsignedIntLE(out, hash); + ByteUtils.writeUnsignedIntLE(out, hash); } bufferOffset = 0; } @@ -200,7 +200,7 @@ private void writeBlock() throws IOException { * @throws IOException */ private void writeEndMark() throws IOException { - Utils.writeUnsignedIntLE(out, 0); + ByteUtils.writeUnsignedIntLE(out, 0); // TODO implement content checksum, update flg.validate() finished = true; } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 32ddcee1acdec..f3cf43c896944 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.ByteBufferOutputStream; import java.io.DataOutputStream; import java.io.IOException; diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java index a54d65dfd8718..99322382b1ee0 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Record.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java @@ -17,6 +17,8 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.ByteBufferOutputStream; +import org.apache.kafka.common.utils.ByteUtils; import org.apache.kafka.common.utils.Crc32; import org.apache.kafka.common.utils.Utils; @@ -118,7 +120,7 @@ public long computeChecksum() { * Retrieve the previously computed CRC for this record */ public long checksum() { - return Utils.readUnsignedInt(buffer, CRC_OFFSET); + return ByteUtils.readUnsignedInt(buffer, CRC_OFFSET); } /** @@ -468,7 +470,7 @@ public static void writeCompressedRecordHeader(ByteBuffer buffer, // compute and fill the crc from the beginning of the message long crc = Utils.computeChecksum(buffer, recordPosition + MAGIC_OFFSET, recordSize - MAGIC_OFFSET); - Utils.writeUnsignedInt(buffer, recordPosition + CRC_OFFSET, crc); + ByteUtils.writeUnsignedInt(buffer, recordPosition + CRC_OFFSET, crc); } private static void write(ByteBuffer buffer, diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java index b6f049c67c351..710ce3bf1a444 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java +++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.utils.AbstractIterator; +import org.apache.kafka.common.utils.ByteBufferInputStream; import org.apache.kafka.common.utils.Utils; import java.io.DataInputStream; diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java b/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferInputStream.java similarity index 97% rename from clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java rename to clients/src/main/java/org/apache/kafka/common/utils/ByteBufferInputStream.java index c033b6c70bec8..094a1a70db01d 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferInputStream.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.record; +package org.apache.kafka.common.utils; import java.io.InputStream; import java.nio.ByteBuffer; diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferOutputStream.java similarity index 97% rename from clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java rename to clients/src/main/java/org/apache/kafka/common/utils/ByteBufferOutputStream.java index 4eee605e96c0a..9480c6de784a0 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ByteBufferOutputStream.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.record; +package org.apache.kafka.common.utils; import java.io.OutputStream; import java.nio.ByteBuffer; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java new file mode 100644 index 0000000000000..50c90a8a814c4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.utils; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +/** + * This classes exposes low-level methods for reading/writing from byte streams or buffers. + */ +public final class ByteUtils { + + private ByteUtils() {} + + /** + * Read an unsigned integer from the given position without modifying the buffers position + * + * @param buffer the buffer to read from + * @param index the index from which to read the integer + * @return The integer read, as a long to avoid signedness + */ + public static long readUnsignedInt(ByteBuffer buffer, int index) { + return buffer.getInt(index) & 0xffffffffL; + } + + /** + * Read an unsigned integer stored in little-endian format from the {@link InputStream}. + * + * @param in The stream to read from + * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) + */ + public static int readUnsignedIntLE(InputStream in) throws IOException { + return in.read() + | (in.read() << 8) + | (in.read() << 16) + | (in.read() << 24); + } + + /** + * Read an unsigned integer stored in little-endian format from a byte array + * at a given offset. + * + * @param buffer The byte array to read from + * @param offset The position in buffer to read from + * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) + */ + public static int readUnsignedIntLE(byte[] buffer, int offset) { + return (buffer[offset] << 0 & 0xff) + | ((buffer[offset + 1] & 0xff) << 8) + | ((buffer[offset + 2] & 0xff) << 16) + | ((buffer[offset + 3] & 0xff) << 24); + } + + /** + * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. + * + * @param buffer The buffer to write to + * @param index The position in the buffer at which to begin writing + * @param value The value to write + */ + public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) { + buffer.putInt(index, (int) (value & 0xffffffffL)); + } + + /** + * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. + * + * @param buffer The buffer to write to + * @param value The value to write + */ + public static void writeUnsignedInt(ByteBuffer buffer, long value) { + buffer.putInt((int) (value & 0xffffffffL)); + } + + /** + * Write an unsigned integer in little-endian format to the {@link OutputStream}. + * + * @param out The stream to write to + * @param value The value to write + */ + public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException { + out.write(value); + out.write(value >>> 8); + out.write(value >>> 16); + out.write(value >>> 24); + } + + /** + * Write an unsigned integer in little-endian format to a byte array + * at a given offset. + * + * @param buffer The byte array to write to + * @param offset The position in buffer to write to + * @param value The value to write + */ + public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { + buffer[offset] = (byte) value; + buffer[offset + 1] = (byte) (value >>> 8); + buffer[offset + 2] = (byte) (value >>> 16); + buffer[offset + 3] = (byte) (value >>> 24); + } + + /** + * Read an integer stored in variable-length format using zig-zag decoding from + *
    Google Protocol Buffers. + * + * @param buffer The buffer to read from + * @return The integer read + * + * @throws IllegalArgumentException if variable-length value does not terminate after 5 bytes have been read + */ + public static int readVarint(ByteBuffer buffer) { + int value = 0; + int i = 0; + int b; + while (((b = buffer.get()) & 0x80) != 0) { + value |= (b & 0x7f) << i; + i += 7; + if (i > 28) + throw illegalVarintException(value); + } + value |= b << i; + return (value >>> 1) ^ -(value & 1); + } + + /** + * Read an integer stored in variable-length format using zig-zag decoding from + * Google Protocol Buffers. + * + * @param in The input to read from + * @return The integer read + * + * @throws IllegalArgumentException if variable-length value does not terminate after 5 bytes have been read + * @throws IOException if {@link DataInput} throws {@link IOException} + */ + public static int readVarint(DataInput in) throws IOException { + int value = 0; + int i = 0; + int b; + while (((b = in.readByte()) & 0x80) != 0) { + value |= (b & 0x7f) << i; + i += 7; + if (i > 28) + throw illegalVarintException(value); + } + value |= b << i; + return (value >>> 1) ^ -(value & 1); + } + + /** + * Read a long stored in variable-length format using zig-zag decoding from + * Google Protocol Buffers. + * + * @param in The input to read from + * @return The long value read + * + * @throws IllegalArgumentException if variable-length value does not terminate after 10 bytes have been read + * @throws IOException if {@link DataInput} throws {@link IOException} + */ + public static long readVarlong(DataInput in) throws IOException { + long value = 0L; + int i = 0; + long b; + while (((b = in.readByte()) & 0x80) != 0) { + value |= (b & 0x7f) << i; + i += 7; + if (i > 63) + throw illegalVarlongException(value); + } + value |= b << i; + return (value >>> 1) ^ -(value & 1); + } + + /** + * Read a long stored in variable-length format using zig-zag decoding from + * Google Protocol Buffers. + * + * @param buffer The buffer to read from + * @return The long value read + * + * @throws IllegalArgumentException if variable-length value does not terminate after 10 bytes have been read + */ + public static long readVarlong(ByteBuffer buffer) { + long value = 0L; + int i = 0; + long b; + while (((b = buffer.get()) & 0x80) != 0) { + value |= (b & 0x7f) << i; + i += 7; + if (i > 63) + throw illegalVarlongException(value); + } + value |= b << i; + return (value >>> 1) ^ -(value & 1); + } + + /** + * Write the given integer following the variable-length zig-zag encoding from + * Google Protocol Buffers + * into the output. + * + * @param value The value to write + * @param out The output to write to + */ + public static void writeVarint(int value, DataOutput out) throws IOException { + int v = (value << 1) ^ (value >> 31); + while ((v & 0xffffff80) != 0L) { + out.writeByte((v & 0x7f) | 0x80); + v >>>= 7; + } + out.writeByte((byte) v); + } + + /** + * Write the given integer following the variable-length zig-zag encoding from + * Google Protocol Buffers + * into the buffer. + * + * @param value The value to write + * @param buffer The output to write to + */ + public static void writeVarint(int value, ByteBuffer buffer) { + int v = (value << 1) ^ (value >> 31); + while ((v & 0xffffff80) != 0L) { + byte b = (byte) ((v & 0x7f) | 0x80); + buffer.put(b); + v >>>= 7; + } + buffer.put((byte) v); + } + + /** + * Write the given integer following the variable-length zig-zag encoding from + * Google Protocol Buffers + * into the output. + * + * @param value The value to write + * @param out The output to write to + */ + public static void writeVarlong(long value, DataOutput out) throws IOException { + long v = (value << 1) ^ (value >> 63); + while ((v & 0xffffffffffffff80L) != 0L) { + out.writeByte(((int) v & 0x7f) | 0x80); + v >>>= 7; + } + out.writeByte((byte) v); + } + + /** + * Write the given integer following the variable-length zig-zag encoding from + * Google Protocol Buffers + * into the buffer. + * + * @param value The value to write + * @param buffer The buffer to write to + */ + public static void writeVarlong(long value, ByteBuffer buffer) { + long v = (value << 1) ^ (value >> 63); + while ((v & 0xffffffffffffff80L) != 0L) { + byte b = (byte) ((v & 0x7f) | 0x80); + buffer.put(b); + v >>>= 7; + } + buffer.put((byte) v); + } + + /** + * Number of bytes needed to encode an integer in variable-length format. + * + * @param value The signed value + */ + public static int sizeOfVarint(int value) { + int v = (value << 1) ^ (value >> 31); + int bytes = 1; + while ((v & 0xffffff80) != 0L) { + bytes += 1; + v >>>= 7; + } + return bytes; + } + + /** + * Number of bytes needed to encode a long in variable-length format. + * + * @param value The signed value + */ + public static int sizeOfVarlong(long value) { + long v = (value << 1) ^ (value >> 63); + int bytes = 1; + while ((v & 0xffffffffffffff80L) != 0L) { + bytes += 1; + v >>>= 7; + } + return bytes; + } + + private static IllegalArgumentException illegalVarintException(int value) { + throw new IllegalArgumentException("Varint is too long, the most significant bit in the 5th byte is set, " + + "converted value: " + Integer.toHexString(value)); + } + + private static IllegalArgumentException illegalVarlongException(long value) { + throw new IllegalArgumentException("Varlong is too long, most significant bit in the 10th byte is set, " + + "converted value: " + Long.toHexString(value)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index ed5eddb192536..6dce97874ae40 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -16,19 +16,24 @@ */ package org.apache.kafka.common.utils; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; +import org.apache.kafka.common.KafkaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.Closeable; +import java.io.EOFException; import java.io.File; import java.io.FileInputStream; -import java.io.OutputStream; -import java.io.UnsupportedEncodingException; import java.io.FileNotFoundException; -import java.io.StringWriter; +import java.io.IOException; +import java.io.InputStream; import java.io.PrintWriter; +import java.io.StringWriter; +import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.Charset; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; @@ -41,16 +46,10 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; -import java.util.Properties; -import java.nio.channels.FileChannel; -import java.nio.charset.Charset; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.kafka.common.KafkaException; public class Utils { @@ -102,85 +101,6 @@ public static byte[] utf8(String string) { } } - /** - * Read an unsigned integer from the given position without modifying the buffers position - * - * @param buffer the buffer to read from - * @param index the index from which to read the integer - * @return The integer read, as a long to avoid signedness - */ - public static long readUnsignedInt(ByteBuffer buffer, int index) { - return buffer.getInt(index) & 0xffffffffL; - } - - /** - * Read an unsigned integer stored in little-endian format from the {@link InputStream}. - * - * @param in The stream to read from - * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) - */ - public static int readUnsignedIntLE(InputStream in) throws IOException { - return in.read() - | (in.read() << 8) - | (in.read() << 16) - | (in.read() << 24); - } - - /** - * Read an unsigned integer stored in little-endian format from a byte array - * at a given offset. - * - * @param buffer The byte array to read from - * @param offset The position in buffer to read from - * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) - */ - public static int readUnsignedIntLE(byte[] buffer, int offset) { - return (buffer[offset] << 0 & 0xff) - | ((buffer[offset + 1] & 0xff) << 8) - | ((buffer[offset + 2] & 0xff) << 16) - | ((buffer[offset + 3] & 0xff) << 24); - } - - /** - * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. - * - * @param buffer The buffer to write to - * @param index The position in the buffer at which to begin writing - * @param value The value to write - */ - public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) { - buffer.putInt(index, (int) (value & 0xffffffffL)); - } - - /** - * Write an unsigned integer in little-endian format to the {@link OutputStream}. - * - * @param out The stream to write to - * @param value The value to write - */ - public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException { - out.write(value); - out.write(value >>> 8); - out.write(value >>> 16); - out.write(value >>> 24); - } - - /** - * Write an unsigned integer in little-endian format to a byte array - * at a given offset. - * - * @param buffer The byte array to write to - * @param offset The position in buffer to write to - * @param value The value to write - */ - public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { - buffer[offset] = (byte) value; - buffer[offset + 1] = (byte) (value >>> 8); - buffer[offset + 2] = (byte) (value >>> 16); - buffer[offset + 3] = (byte) (value >>> 24); - } - - /** * Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from * java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). @@ -824,4 +744,5 @@ public static void readFully(FileChannel channel, ByteBuffer destinationBuffer, currentPosition += bytesRead; } while (bytesRead != -1 && destinationBuffer.hasRemaining()); } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 3f65caf244d52..de527f8db12db 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -40,7 +40,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.record.ByteBufferOutputStream; +import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index 1c14e829a3005..136b55aaea267 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -40,6 +40,8 @@ public void setup() { new Field("int16", Type.INT16), new Field("int32", Type.INT32), new Field("int64", Type.INT64), + new Field("varint", Type.VARINT), + new Field("varlong", Type.VARLONG), new Field("string", Type.STRING), new Field("nullable_string", Type.NULLABLE_STRING), new Field("bytes", Type.BYTES), @@ -52,6 +54,8 @@ public void setup() { .set("int16", (short) 1) .set("int32", 1) .set("int64", 1L) + .set("varint", 300) + .set("varlong", 500L) .set("string", "1") .set("nullable_string", null) .set("bytes", ByteBuffer.wrap("1".getBytes())) @@ -80,6 +84,10 @@ public void testSimple() { check(Type.NULLABLE_BYTES, null); check(Type.NULLABLE_BYTES, ByteBuffer.allocate(0)); check(Type.NULLABLE_BYTES, ByteBuffer.wrap("abcd".getBytes())); + check(Type.VARINT, Integer.MAX_VALUE); + check(Type.VARINT, Integer.MIN_VALUE); + check(Type.VARLONG, Long.MAX_VALUE); + check(Type.VARLONG, Long.MIN_VALUE); check(new ArrayOf(Type.INT32), new Object[] {1, 2, 3, 4}); check(new ArrayOf(Type.STRING), new Object[] {}); check(new ArrayOf(Type.STRING), new Object[] {"hello", "there", "beautiful"}); diff --git a/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java index 5a1d5f4d10232..21cbfb7ccfcac 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/CompressionTypeTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.record; +import org.apache.kafka.common.utils.ByteBufferInputStream; +import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.junit.Test; import java.nio.ByteBuffer; diff --git a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java index 4bb90cddfd716..aa77ca408263c 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.record; +import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.Utils; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java new file mode 100644 index 0000000000000..0a082fb48323c --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.utils; + +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +public class ByteUtilsTest { + private final byte x00 = 0x00; + private final byte x01 = 0x01; + private final byte x02 = 0x02; + private final byte x0F = 0x0f; + private final byte x7E = 0x7E; + private final byte x7F = 0x7F; + private final byte xFF = (byte) 0xff; + private final byte x80 = (byte) 0x80; + private final byte x81 = (byte) 0x81; + private final byte xFE = (byte) 0xfe; + + @Test + public void testReadUnsignedIntLEFromArray() { + byte[] array1 = {0x01, 0x02, 0x03, 0x04, 0x05}; + assertEquals(0x04030201, ByteUtils.readUnsignedIntLE(array1, 0)); + assertEquals(0x05040302, ByteUtils.readUnsignedIntLE(array1, 1)); + + byte[] array2 = {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, (byte) 0xf5, (byte) 0xf6}; + assertEquals(0xf4f3f2f1, ByteUtils.readUnsignedIntLE(array2, 0)); + assertEquals(0xf6f5f4f3, ByteUtils.readUnsignedIntLE(array2, 2)); + } + + @Test + public void testReadUnsignedIntLEFromInputStream() throws IOException { + byte[] array1 = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09}; + ByteArrayInputStream is1 = new ByteArrayInputStream(array1); + assertEquals(0x04030201, ByteUtils.readUnsignedIntLE(is1)); + assertEquals(0x08070605, ByteUtils.readUnsignedIntLE(is1)); + + byte[] array2 = {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, (byte) 0xf5, (byte) 0xf6, (byte) 0xf7, (byte) 0xf8}; + ByteArrayInputStream is2 = new ByteArrayInputStream(array2); + assertEquals(0xf4f3f2f1, ByteUtils.readUnsignedIntLE(is2)); + assertEquals(0xf8f7f6f5, ByteUtils.readUnsignedIntLE(is2)); + } + + @Test + public void testWriteUnsignedIntLEToArray() { + int value1 = 0x04030201; + + byte[] array1 = new byte[4]; + ByteUtils.writeUnsignedIntLE(array1, 0, value1); + assertArrayEquals(new byte[] {0x01, 0x02, 0x03, 0x04}, array1); + + array1 = new byte[8]; + ByteUtils.writeUnsignedIntLE(array1, 2, value1); + assertArrayEquals(new byte[] {0, 0, 0x01, 0x02, 0x03, 0x04, 0, 0}, array1); + + int value2 = 0xf4f3f2f1; + + byte[] array2 = new byte[4]; + ByteUtils.writeUnsignedIntLE(array2, 0, value2); + assertArrayEquals(new byte[] {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4}, array2); + + array2 = new byte[8]; + ByteUtils.writeUnsignedIntLE(array2, 2, value2); + assertArrayEquals(new byte[] {0, 0, (byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, 0, 0}, array2); + } + + @Test + public void testWriteUnsignedIntLEToOutputStream() throws IOException { + int value1 = 0x04030201; + ByteArrayOutputStream os1 = new ByteArrayOutputStream(); + ByteUtils.writeUnsignedIntLE(os1, value1); + ByteUtils.writeUnsignedIntLE(os1, value1); + assertArrayEquals(new byte[] {0x01, 0x02, 0x03, 0x04, 0x01, 0x02, 0x03, 0x04}, os1.toByteArray()); + + int value2 = 0xf4f3f2f1; + ByteArrayOutputStream os2 = new ByteArrayOutputStream(); + ByteUtils.writeUnsignedIntLE(os2, value2); + assertArrayEquals(new byte[] {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4}, os2.toByteArray()); + } + + @Test + public void testVarintSerde() throws Exception { + assertVarintSerde(0, new byte[] {x00}); + assertVarintSerde(-1, new byte[] {x01}); + assertVarintSerde(1, new byte[] {x02}); + assertVarintSerde(63, new byte[] {x7E}); + assertVarintSerde(-64, new byte[] {x7F}); + assertVarintSerde(64, new byte[] {x80, x01}); + assertVarintSerde(-65, new byte[] {x81, x01}); + assertVarintSerde(8191, new byte[] {xFE, x7F}); + assertVarintSerde(-8192, new byte[] {xFF, x7F}); + assertVarintSerde(8192, new byte[] {x80, x80, x01}); + assertVarintSerde(-8193, new byte[] {x81, x80, x01}); + assertVarintSerde(1048575, new byte[] {xFE, xFF, x7F}); + assertVarintSerde(-1048576, new byte[] {xFF, xFF, x7F}); + assertVarintSerde(1048576, new byte[] {x80, x80, x80, x01}); + assertVarintSerde(-1048577, new byte[] {x81, x80, x80, x01}); + assertVarintSerde(134217727, new byte[] {xFE, xFF, xFF, x7F}); + assertVarintSerde(-134217728, new byte[] {xFF, xFF, xFF, x7F}); + assertVarintSerde(134217728, new byte[] {x80, x80, x80, x80, x01}); + assertVarintSerde(-134217729, new byte[] {x81, x80, x80, x80, x01}); + assertVarintSerde(Integer.MAX_VALUE, new byte[] {xFE, xFF, xFF, xFF, x0F}); + assertVarintSerde(Integer.MIN_VALUE, new byte[] {xFF, xFF, xFF, xFF, x0F}); + } + + @Test + public void testVarlongSerde() throws Exception { + assertVarlongSerde(0, new byte[] {x00}); + assertVarlongSerde(-1, new byte[] {x01}); + assertVarlongSerde(1, new byte[] {x02}); + assertVarlongSerde(63, new byte[] {x7E}); + assertVarlongSerde(-64, new byte[] {x7F}); + assertVarlongSerde(64, new byte[] {x80, x01}); + assertVarlongSerde(-65, new byte[] {x81, x01}); + assertVarlongSerde(8191, new byte[] {xFE, x7F}); + assertVarlongSerde(-8192, new byte[] {xFF, x7F}); + assertVarlongSerde(8192, new byte[] {x80, x80, x01}); + assertVarlongSerde(-8193, new byte[] {x81, x80, x01}); + assertVarlongSerde(1048575, new byte[] {xFE, xFF, x7F}); + assertVarlongSerde(-1048576, new byte[] {xFF, xFF, x7F}); + assertVarlongSerde(1048576, new byte[] {x80, x80, x80, x01}); + assertVarlongSerde(-1048577, new byte[] {x81, x80, x80, x01}); + assertVarlongSerde(134217727, new byte[] {xFE, xFF, xFF, x7F}); + assertVarlongSerde(-134217728, new byte[] {xFF, xFF, xFF, x7F}); + assertVarlongSerde(134217728, new byte[] {x80, x80, x80, x80, x01}); + assertVarlongSerde(-134217729, new byte[] {x81, x80, x80, x80, x01}); + assertVarlongSerde(Integer.MAX_VALUE, new byte[] {xFE, xFF, xFF, xFF, x0F}); + assertVarlongSerde(Integer.MIN_VALUE, new byte[] {xFF, xFF, xFF, xFF, x0F}); + assertVarlongSerde(17179869183L, new byte[] {xFE, xFF, xFF, xFF, x7F}); + assertVarlongSerde(-17179869184L, new byte[] {xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(17179869184L, new byte[] {x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(-17179869185L, new byte[] {x81, x80, x80, x80, x80, x01}); + assertVarlongSerde(2199023255551L, new byte[] {xFE, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(-2199023255552L, new byte[] {xFF, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(2199023255552L, new byte[] {x80, x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(-2199023255553L, new byte[] {x81, x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(281474976710655L, new byte[] {xFE, xFF, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(-281474976710656L, new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(281474976710656L, new byte[] {x80, x80, x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(-281474976710657L, new byte[] {x81, x80, x80, x80, x80, x80, x80, 1}); + assertVarlongSerde(36028797018963967L, new byte[] {xFE, xFF, xFF, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(-36028797018963968L, new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(36028797018963968L, new byte[] {x80, x80, x80, x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(-36028797018963969L, new byte[] {x81, x80, x80, x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(4611686018427387903L, new byte[] {xFE, xFF, xFF, xFF, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(-4611686018427387904L, new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, x7F}); + assertVarlongSerde(4611686018427387904L, new byte[] {x80, x80, x80, x80, x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(-4611686018427387905L, new byte[] {x81, x80, x80, x80, x80, x80, x80, x80, x80, x01}); + assertVarlongSerde(Long.MAX_VALUE, new byte[] {xFE, xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, x01}); + assertVarlongSerde(Long.MIN_VALUE, new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, x01}); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidVarint() { + // varint encoding has one overflow byte + ByteBuffer buf = ByteBuffer.wrap(new byte[] {xFF, xFF, xFF, xFF, xFF, x01}); + ByteUtils.readVarint(buf); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidVarlong() { + // varlong encoding has one overflow byte + ByteBuffer buf = ByteBuffer.wrap(new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF, x01}); + ByteUtils.readVarlong(buf); + } + + private void assertVarintSerde(int value, byte[] expectedEncoding) throws IOException { + ByteBuffer buf = ByteBuffer.allocate(32); + ByteUtils.writeVarint(value, buf); + buf.flip(); + assertArrayEquals(expectedEncoding, Utils.toArray(buf)); + assertEquals(value, ByteUtils.readVarint(buf.duplicate())); + + buf.rewind(); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buf)); + ByteUtils.writeVarint(value, out); + buf.flip(); + assertArrayEquals(expectedEncoding, Utils.toArray(buf)); + DataInputStream in = new DataInputStream(new ByteBufferInputStream(buf)); + assertEquals(value, ByteUtils.readVarint(in)); + } + + private void assertVarlongSerde(long value, byte[] expectedEncoding) throws IOException { + ByteBuffer buf = ByteBuffer.allocate(32); + ByteUtils.writeVarlong(value, buf); + buf.flip(); + assertEquals(value, ByteUtils.readVarlong(buf.duplicate())); + assertArrayEquals(expectedEncoding, Utils.toArray(buf)); + + buf.rewind(); + DataOutputStream out = new DataOutputStream(new ByteBufferOutputStream(buf)); + ByteUtils.writeVarlong(value, out); + buf.flip(); + assertArrayEquals(expectedEncoding, Utils.toArray(buf)); + DataInputStream in = new DataInputStream(new ByteBufferInputStream(buf)); + assertEquals(value, ByteUtils.readVarlong(in)); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 5f36c1cfcd973..7672335d3e292 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -16,28 +16,24 @@ */ package org.apache.kafka.common.utils; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; +import org.apache.kafka.test.TestUtils; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.junit.Test; + +import java.io.Closeable; import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; import java.util.Arrays; import java.util.Collections; -import java.io.Closeable; -import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Random; -import org.apache.kafka.test.TestUtils; -import org.easymock.EasyMock; -import org.easymock.IAnswer; -import org.junit.Test; - - import static org.apache.kafka.common.utils.Utils.formatAddress; import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getPort; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -337,64 +333,4 @@ static void checkException(IOException e, TestCloseable... closeablesWithExcepti } } - @Test - public void testReadUnsignedIntLEFromArray() { - byte[] array1 = {0x01, 0x02, 0x03, 0x04, 0x05}; - assertEquals(0x04030201, Utils.readUnsignedIntLE(array1, 0)); - assertEquals(0x05040302, Utils.readUnsignedIntLE(array1, 1)); - - byte[] array2 = {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, (byte) 0xf5, (byte) 0xf6}; - assertEquals(0xf4f3f2f1, Utils.readUnsignedIntLE(array2, 0)); - assertEquals(0xf6f5f4f3, Utils.readUnsignedIntLE(array2, 2)); - } - - @Test - public void testReadUnsignedIntLEFromInputStream() throws IOException { - byte[] array1 = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09}; - ByteArrayInputStream is1 = new ByteArrayInputStream(array1); - assertEquals(0x04030201, Utils.readUnsignedIntLE(is1)); - assertEquals(0x08070605, Utils.readUnsignedIntLE(is1)); - - byte[] array2 = {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, (byte) 0xf5, (byte) 0xf6, (byte) 0xf7, (byte) 0xf8}; - ByteArrayInputStream is2 = new ByteArrayInputStream(array2); - assertEquals(0xf4f3f2f1, Utils.readUnsignedIntLE(is2)); - assertEquals(0xf8f7f6f5, Utils.readUnsignedIntLE(is2)); - } - - @Test - public void testWriteUnsignedIntLEToArray() { - int value1 = 0x04030201; - - byte[] array1 = new byte[4]; - Utils.writeUnsignedIntLE(array1, 0, value1); - assertArrayEquals(new byte[] {0x01, 0x02, 0x03, 0x04}, array1); - - array1 = new byte[8]; - Utils.writeUnsignedIntLE(array1, 2, value1); - assertArrayEquals(new byte[] {0, 0, 0x01, 0x02, 0x03, 0x04, 0, 0}, array1); - - int value2 = 0xf4f3f2f1; - - byte[] array2 = new byte[4]; - Utils.writeUnsignedIntLE(array2, 0, value2); - assertArrayEquals(new byte[] {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4}, array2); - - array2 = new byte[8]; - Utils.writeUnsignedIntLE(array2, 2, value2); - assertArrayEquals(new byte[] {0, 0, (byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4, 0, 0}, array2); - } - - @Test - public void testWriteUnsignedIntLEToOutputStream() throws IOException { - int value1 = 0x04030201; - ByteArrayOutputStream os1 = new ByteArrayOutputStream(); - Utils.writeUnsignedIntLE(os1, value1); - Utils.writeUnsignedIntLE(os1, value1); - assertArrayEquals(new byte[] {0x01, 0x02, 0x03, 0x04, 0x01, 0x02, 0x03, 0x04}, os1.toByteArray()); - - int value2 = 0xf4f3f2f1; - ByteArrayOutputStream os2 = new ByteArrayOutputStream(); - Utils.writeUnsignedIntLE(os2, value2); - assertArrayEquals(new byte[] {(byte) 0xf1, (byte) 0xf2, (byte) 0xf3, (byte) 0xf4}, os2.toByteArray()); - } } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index e0efb3d398380..474f934dd8538 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -23,7 +23,8 @@ import org.apache.kafka.common.record.{Record, TimestampType} import scala.math._ import kafka.utils._ -import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.utils.ByteUtils.{readUnsignedInt, writeUnsignedInt} +import org.apache.kafka.common.utils.{ByteUtils, Utils} /** * Constants related to messages @@ -201,7 +202,7 @@ class Message(val buffer: ByteBuffer, buffer.rewind() // now compute the checksum and fill it in - Utils.writeUnsignedInt(buffer, CrcOffset, computeChecksum) + ByteUtils.writeUnsignedInt(buffer, CrcOffset, computeChecksum) } def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) = @@ -228,7 +229,7 @@ class Message(val buffer: ByteBuffer, /** * Retrieve the previously computed CRC for this message */ - def checksum: Long = Utils.readUnsignedInt(buffer, CrcOffset) + def checksum: Long = ByteUtils.readUnsignedInt(buffer, CrcOffset) /** * Returns true if the crc stored with the message matches the crc computed off the message contents diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 46c25afe49d1d..75a86d26c0ab3 100755 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -19,6 +19,7 @@ package kafka.message import java.nio._ import java.util.HashMap + import org.apache.kafka.common.protocol.Errors import scala.collection._ @@ -26,7 +27,7 @@ import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{Before, Test} import kafka.utils.TestUtils -import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.utils.ByteUtils case class MessageTestVal(key: Array[Byte], payload: Array[Byte], @@ -89,7 +90,7 @@ class MessageTest extends JUnitSuite { assertTrue("Auto-computed checksum should be valid", v.message.isValid) // garble checksum val badChecksum: Int = (v.message.checksum + 1 % Int.MaxValue).toInt - Utils.writeUnsignedInt(v.message.buffer, Message.CrcOffset, badChecksum) + ByteUtils.writeUnsignedInt(v.message.buffer, Message.CrcOffset, badChecksum) assertFalse("Message with invalid checksum should be invalid", v.message.isValid) } } diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index f279c856408f0..a5d810286af04 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -23,7 +23,7 @@ import kafka.server.AbstractFetcherThread.{FetchRequest, PartitionData} import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.utils.ByteUtils import org.apache.kafka.common.record.{MemoryRecords, Record} import org.junit.Assert.{assertFalse, assertTrue} import org.junit.{Before, Test} @@ -184,7 +184,7 @@ class AbstractFetcherThreadTest { val corruptedRecord = Record.create("hello".getBytes()) val badChecksum = (corruptedRecord.checksum + 1 % Int.MaxValue).toInt // Garble checksum - Utils.writeUnsignedInt(corruptedRecord.buffer, Record.CRC_OFFSET, badChecksum) + ByteUtils.writeUnsignedInt(corruptedRecord.buffer, Record.CRC_OFFSET, badChecksum) val records = MemoryRecords.withRecords(corruptedRecord) fetchRequest.offsets.mapValues(_ => new TestPartitionData(records)).toSeq } else diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index f35de402fead1..77fb58a113cb0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.common.record.ByteBufferInputStream; +import org.apache.kafka.common.utils.ByteBufferInputStream; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.TaskId; From 022bf129518e33e165f9ceefc4ab9e622952d3bd Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 8 Mar 2017 23:16:53 +0000 Subject: [PATCH 080/101] MINOR: Use ConcurrentMap for ConsumerNetworkClient UnsentRequests Author: Jason Gustafson Reviewers: Ismael Juma Closes #2656 from hachikuji/minor-cleanup-unsent-requests --- .../kafka/clients/consumer/KafkaConsumer.java | 2 +- .../internals/ConsumerNetworkClient.java | 95 +++++++++++-------- .../clients/consumer/internals/Fetcher.java | 2 +- 3 files changed, 58 insertions(+), 41 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 51b00afd42d05..612f446ab9d2b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1004,7 +1004,7 @@ public ConsumerRecords poll(long timeout) { // // NOTE: since the consumed position has already been updated, we must not allow // wakeups or any other errors to be triggered prior to returning the fetched records. - if (fetcher.sendFetches() > 0 || client.hasPendingRequest()) + if (fetcher.sendFetches() > 0 || client.hasPendingRequests()) client.pollNoWakeup(); if (this.interceptors == null) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index eb25359a05d42..478ed3f7d0024 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -37,11 +37,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; -import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.kafka.common.errors.InterruptException; @@ -242,6 +242,9 @@ public void poll(long timeout, long now, PollCondition pollCondition) { // fail requests that couldn't be sent if they have expired failExpiredRequests(now); + + // clean unsent requests collection to keep the map from growing indefinitely + unsent.clean(); } // called without the lock to avoid deadlock potential if handlers need to acquire locks @@ -271,12 +274,12 @@ public boolean awaitPendingRequests(Node node, long timeoutMs) { long startMs = time.milliseconds(); long remainingMs = timeoutMs; - while (hasPendingRequest(node) && remainingMs > 0) { + while (hasPendingRequests(node) && remainingMs > 0) { poll(remainingMs); remainingMs = timeoutMs - (time.milliseconds() - startMs); } - return !hasPendingRequest(node); + return !hasPendingRequests(node); } /** @@ -297,8 +300,8 @@ public int pendingRequestCount(Node node) { * @param node The node in question * @return A boolean indicating whether there is pending request */ - public boolean hasPendingRequest(Node node) { - if (unsent.hasRequest(node)) + public boolean hasPendingRequests(Node node) { + if (unsent.hasRequests(node)) return true; synchronized (this) { return client.inFlightRequestCount(node.idString()) > 0; @@ -321,8 +324,8 @@ public int pendingRequestCount() { * have been transmitted (i.e. in-flight requests) and those which are awaiting transmission. * @return A boolean indicating whether there is pending request */ - public boolean hasPendingRequest() { - if (unsent.hasRequest()) + public boolean hasPendingRequests() { + if (unsent.hasRequests()) return true; synchronized (this) { return client.inFlightRequestCount() > 0; @@ -350,8 +353,7 @@ private void checkDisconnects(long now) { // by NetworkClient, so we just need to check whether connections for any of the unsent // requests have been disconnected; if they have, then we complete the corresponding future // and set the disconnect flag in the ClientResponse - List nodes = unsent.nodes(); - for (Node node : nodes) { + for (Node node : unsent.nodes()) { if (client.connectionFailed(node)) { // Remove entry before invoking request callback to avoid callbacks handling // coordinator failures traversing the unsent list again. @@ -368,7 +370,7 @@ private void checkDisconnects(long now) { private void failExpiredRequests(long now) { // clear all expired unsent requests and fail their corresponding futures - List expiredRequests = unsent.removeExpiredRequests(now, unsentExpiryMs); + Collection expiredRequests = unsent.removeExpiredRequests(now, unsentExpiryMs); for (ClientRequest request : expiredRequests) { RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); handler.onFailure(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); @@ -471,12 +473,12 @@ public void tryConnect(Node node) { } } - public class RequestFutureCompletionHandler implements RequestCompletionHandler { + private class RequestFutureCompletionHandler implements RequestCompletionHandler { private final RequestFuture future; private ClientResponse response; private RuntimeException e; - public RequestFutureCompletionHandler() { + private RequestFutureCompletionHandler() { this.future = new RequestFuture<>(); } @@ -525,55 +527,55 @@ public interface PollCondition { boolean shouldBlock(); } - /* * A threadsafe helper class to hold requests per node that have not been sent yet */ private final static class UnsentRequests { - private final Map> unsent; + private final ConcurrentMap> unsent; - public UnsentRequests() { - unsent = new HashMap<>(); + private UnsentRequests() { + unsent = new ConcurrentHashMap<>(); } - public synchronized void put(Node node, ClientRequest request) { - ConcurrentLinkedQueue nodeUnsent = unsent.get(node); - if (nodeUnsent == null) { - nodeUnsent = new ConcurrentLinkedQueue<>(); - unsent.put(node, nodeUnsent); + public void put(Node node, ClientRequest request) { + // the lock protects the put from a concurrent removal of the queue for the node + synchronized (unsent) { + ConcurrentLinkedQueue requests = unsent.get(node); + if (requests == null) { + requests = new ConcurrentLinkedQueue<>(); + unsent.put(node, requests); + } + requests.add(request); } - nodeUnsent.add(request); } - public synchronized int requestCount(Node node) { + public int requestCount(Node node) { ConcurrentLinkedQueue requests = unsent.get(node); return requests == null ? 0 : requests.size(); } - public synchronized int requestCount() { + public int requestCount() { int total = 0; for (ConcurrentLinkedQueue requests : unsent.values()) total += requests.size(); return total; } - public synchronized boolean hasRequest(Node node) { + public boolean hasRequests(Node node) { ConcurrentLinkedQueue requests = unsent.get(node); return requests != null && !requests.isEmpty(); } - public synchronized boolean hasRequest() { + public boolean hasRequests() { for (ConcurrentLinkedQueue requests : unsent.values()) if (!requests.isEmpty()) return true; return false; } - public synchronized List removeExpiredRequests(long now, long unsentExpiryMs) { + public Collection removeExpiredRequests(long now, long unsentExpiryMs) { List expiredRequests = new ArrayList<>(); - Iterator> iterator = unsent.values().iterator(); - while (iterator.hasNext()) { - ConcurrentLinkedQueue requests = iterator.next(); + for (ConcurrentLinkedQueue requests : unsent.values()) { Iterator requestIterator = requests.iterator(); while (requestIterator.hasNext()) { ClientRequest request = requestIterator.next(); @@ -583,24 +585,39 @@ public synchronized List removeExpiredRequests(long now, long uns } else break; } - if (requests.isEmpty()) - iterator.remove(); } return expiredRequests; } - public synchronized Collection remove(Node node) { - ConcurrentLinkedQueue requests = unsent.remove(node); - return requests == null ? Collections.emptyList() : requests; + public void clean() { + // the lock protects removal from a concurrent put which could otherwise mutate the + // queue after it has been removed from the map + synchronized (unsent) { + Iterator> iterator = unsent.values().iterator(); + while (iterator.hasNext()) { + ConcurrentLinkedQueue requests = iterator.next(); + if (requests.isEmpty()) + iterator.remove(); + } + } + } + + public Collection remove(Node node) { + // the lock protects removal from a concurrent put which could otherwise mutate the + // queue after it has been removed from the map + synchronized (unsent) { + ConcurrentLinkedQueue requests = unsent.remove(node); + return requests == null ? Collections.emptyList() : requests; + } } - public synchronized Iterator requestIterator(Node node) { + public Iterator requestIterator(Node node) { ConcurrentLinkedQueue requests = unsent.get(node); return requests == null ? Collections.emptyIterator() : requests.iterator(); } - public synchronized List nodes() { - return new ArrayList<>(unsent.keySet()); + public Collection nodes() { + return unsent.keySet(); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 536e4e823768b..441206aa5575f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -713,7 +713,7 @@ private Map createFetchRequests() { Node node = cluster.leaderFor(partition); if (node == null) { metadata.requestUpdate(); - } else if (!this.client.hasPendingRequest(node)) { + } else if (!this.client.hasPendingRequests(node)) { // if there is a leader and no in-flight requests, issue a new fetch LinkedHashMap fetch = fetchable.get(node); if (fetch == null) { From 537f98a5d64b86022ac2125c75d079c26e4b9830 Mon Sep 17 00:00:00 2001 From: Will Droste Date: Thu, 9 Mar 2017 01:00:33 +0000 Subject: [PATCH 081/101] KAFKA-4745; Remove unnecessary flush in KafkaLZ4BlockOutputStream.close() Remove unnecessary 'flush', the underlying stream should handle it on close. Author: Will Droste Reviewers: Ismael Juma Closes #2528 from wdroste/trunk --- .../apache/kafka/common/record/KafkaLZ4BlockOutputStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java index 034b94521f60b..9a100f1fa00b9 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java @@ -258,10 +258,10 @@ private void ensureNotFinished() { @Override public void close() throws IOException { if (!finished) { + // basicallly flush the buffer writing the last block writeBlock(); + // write the end block and finish the stream writeEndMark(); - flush(); - finished = true; } if (out != null) { out.close(); From 294018a578cbbc187ac123a6b99990468186e349 Mon Sep 17 00:00:00 2001 From: simplesteph Date: Wed, 8 Mar 2017 21:41:38 -0800 Subject: [PATCH 082/101] KAFKA-4864; added correct zookeeper nodes for security migrator Author: simplesteph Reviewers: Jun Rao Closes #2655 from simplesteph/fix-security-migrator-tool --- .../main/scala/kafka/admin/AdminUtils.scala | 2 +- .../kafka/admin/ZkSecurityMigrator.scala | 4 +- .../security/auth/SimpleAclAuthorizer.scala | 4 +- .../kafka/server/DynamicConfigManager.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 56 +++++++++++-------- ...aslScramSslEndToEndAuthorizationTest.scala | 2 +- .../unit/kafka/admin/TopicCommandTest.scala | 2 +- .../security/auth/ZkAuthorizationTest.scala | 6 +- .../server/DynamicConfigChangeTest.scala | 2 +- 9 files changed, 46 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 65ac91cf361b4..d4ae4ff946614 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -565,7 +565,7 @@ object AdminUtils extends Logging with AdminUtilities { writeEntityConfig(zkUtils, entityConfigPath, configs) // create the change notification - val seqNode = ZkUtils.EntityConfigChangesPath + "/" + EntityConfigChangeZnodePrefix + val seqNode = ZkUtils.ConfigChangesPath + "/" + EntityConfigChangeZnodePrefix val content = Json.encode(getConfigChangeZnodeData(sanitizedEntityPath)) zkUtils.zkClient.createPersistentSequential(seqNode, content) } diff --git a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala index 9bd321c408da4..eb5c14211fc2d 100644 --- a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala +++ b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala @@ -220,12 +220,12 @@ class ZkSecurityMigrator(zkUtils: ZkUtils) extends Logging { private def run(): Unit = { try { setAclIndividually("/") - for (path <- zkUtils.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths) { debug("Going to set ACL for %s".format(path)) zkUtils.makeSurePersistentPathExists(path) setAclsRecursively(path) } - + @tailrec def recurse(): Unit = { val future = futures.synchronized { diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index 7ae47961e73c6..51de3bc5e203f 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -56,10 +56,10 @@ object SimpleAclAuthorizer { * /kafka-acl/Group/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]} *

    */ - val AclZkPath = "/kafka-acl" + val AclZkPath = ZkUtils.KafkaAclPath //notification node which gets updated with the resource name when acl on a resource is changed. - val AclChangedZkPath = "/kafka-acl-changes" + val AclChangedZkPath = ZkUtils.KafkaAclChangesPath //prefix of all the change notification sequence node. val AclChangedPrefix = "acl_changes_" diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index e0e6a03cec410..c81ce6c5e3551 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -148,7 +148,7 @@ class DynamicConfigManager(private val zkUtils: ZkUtils, } } - private val configChangeListener = new ZkNodeChangeNotificationListener(zkUtils, ZkUtils.EntityConfigChangesPath, AdminUtils.EntityConfigChangeZnodePrefix, ConfigChangedNotificationHandler) + private val configChangeListener = new ZkNodeChangeNotificationListener(zkUtils, ZkUtils.ConfigChangesPath, AdminUtils.EntityConfigChangeZnodePrefix, ConfigChangedNotificationHandler) /** * Begin watching for config changes diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 7a6bd6336b886..e67e264b9bd69 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -43,19 +43,40 @@ import scala.collection._ import scala.collection.JavaConverters._ object ZkUtils { - val ConsumersPath = "/consumers" - val ClusterIdPath = "/cluster/id" - val BrokerIdsPath = "/brokers/ids" - val BrokerTopicsPath = "/brokers/topics" + + + // Important: it is necessary to add any new top level Zookeeper path here + val AdminPath = "/admin" + val BrokersPath = "/brokers" + val ClusterPath = "/cluster" + val ConfigPath = "/config" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" - val ReassignPartitionsPath = "/admin/reassign_partitions" - val DeleteTopicsPath = "/admin/delete_topics" - val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" - val BrokerSequenceIdPath = "/brokers/seqid" val IsrChangeNotificationPath = "/isr_change_notification" - val EntityConfigPath = "/config" - val EntityConfigChangesPath = "/config/changes" + val KafkaAclPath = "/kafka-acl" + val KafkaAclChangesPath = "/kafka-acl-changes" + + val ConsumersPath = "/consumers" + val ClusterIdPath = s"$ClusterPath/id" + val BrokerIdsPath = s"$BrokersPath/ids" + val BrokerTopicsPath = s"$BrokersPath/topics" + val ReassignPartitionsPath = s"$AdminPath/reassign_partitions" + val DeleteTopicsPath = s"$AdminPath/delete_topics" + val PreferredReplicaLeaderElectionPath = s"$AdminPath/preferred_replica_election" + val BrokerSequenceIdPath = s"$BrokersPath/seqid" + val ConfigChangesPath = s"$ConfigPath/changes" + + + // Important: it is necessary to add any new top level Zookeeper path to the Seq + val SecureZkRootPaths = Seq(AdminPath, + BrokersPath, + ClusterPath, + ConfigPath, + ControllerPath, + ControllerEpochPath, + IsrChangeNotificationPath, + KafkaAclPath, + KafkaAclChangesPath) def apply(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int, isZkSecurityEnabled: Boolean): ZkUtils = { val (zkClient, zkConnection) = createZkClientAndConnection(zkUrl, sessionTimeout, connectionTimeout) @@ -117,13 +138,13 @@ object ZkUtils { getTopicPartitionPath(topic, partitionId) + "/" + "state" def getEntityConfigRootPath(entityType: String): String = - ZkUtils.EntityConfigPath + "/" + entityType + ZkUtils.ConfigPath + "/" + entityType def getEntityConfigPath(entityType: String, entity: String): String = getEntityConfigRootPath(entityType) + "/" + entity def getEntityConfigPath(entityPath: String): String = - ZkUtils.EntityConfigPath + "/" + entityPath + ZkUtils.ConfigPath + "/" + entityPath def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic @@ -191,22 +212,13 @@ class ZkUtils(val zkClient: ZkClient, val persistentZkPaths = Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, - EntityConfigChangesPath, + ConfigChangesPath, getEntityConfigRootPath(ConfigType.Topic), getEntityConfigRootPath(ConfigType.Client), DeleteTopicsPath, BrokerSequenceIdPath, IsrChangeNotificationPath) - val securePersistentZkPaths = Seq(BrokerIdsPath, - BrokerTopicsPath, - EntityConfigChangesPath, - getEntityConfigRootPath(ConfigType.Topic), - getEntityConfigRootPath(ConfigType.Client), - DeleteTopicsPath, - BrokerSequenceIdPath, - IsrChangeNotificationPath) - val DefaultAcls: java.util.List[ACL] = ZkUtils.DefaultAcls(isSecure) def getController(): Int = { diff --git a/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala index fe0204ae51932..86db4074f9476 100644 --- a/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala @@ -32,7 +32,7 @@ class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTes override def configureSecurityBeforeServersStart() { super.configureSecurityBeforeServersStart() - zkUtils.makeSurePersistentPathExists(ZkUtils.EntityConfigChangesPath) + zkUtils.makeSurePersistentPathExists(ZkUtils.ConfigChangesPath) def configCommandArgs(username: String, password: String) : Array[String] = { val credentials = kafkaServerSaslMechanisms.map(m => s"$m=[iterations=4096,password=$password]") diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index 8ce7c90436f44..52158673e435e 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -49,7 +49,7 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging with RackAwareT assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) // pre-create the topic config changes path to avoid a NoNodeException - zkUtils.createPersistentPath(EntityConfigChangesPath) + zkUtils.createPersistentPath(ConfigChangesPath) // modify the topic to add new partitions val numPartitionsModified = 3 diff --git a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala index 8cec0c79c8a4f..3b4c48e325d70 100644 --- a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala @@ -153,7 +153,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { @Test def testDeleteRecursive() { info(s"zkConnect string: $zkConnect") - for (path <- zkUtils.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths) { info(s"Creating $path") zkUtils.makeSurePersistentPathExists(path) zkUtils.createPersistentPath(s"$path/fpjwashere", "") @@ -185,7 +185,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { */ private def testMigration(zkUrl: String, firstZk: ZkUtils, secondZk: ZkUtils) { info(s"zkConnect string: $zkUrl") - for (path <- firstZk.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths) { info(s"Creating $path") firstZk.makeSurePersistentPathExists(path) // Create a child for each znode to exercise the recurrent @@ -206,7 +206,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { } ZkSecurityMigrator.run(Array(s"--zookeeper.acl=$secureOpt", s"--zookeeper.connect=$zkUrl")) info("Done with migration") - for (path <- secondZk.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths) { val listParent = secondZk.zkConnection.getAcl(path).getKey assertTrue(path, isAclCorrect(listParent, secondZk.isSecure)) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index cf0dc6fd6cdc1..dc30fb20eb0cd 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -148,7 +148,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { AdminUtils.changeUserOrUserClientIdConfig(zkUtils, "ANONYMOUS/clients/overriddenUserClientId", userClientIdProps) // Remove config change znodes to force quota initialization only through loading of user/client quotas - zkUtils.getChildren(ZkUtils.EntityConfigChangesPath).foreach { p => zkUtils.deletePath(ZkUtils.EntityConfigChangesPath + "/" + p) } + zkUtils.getChildren(ZkUtils.ConfigChangesPath).foreach { p => zkUtils.deletePath(ZkUtils.ConfigChangesPath + "/" + p) } server.startup() val quotaManagers = server.apis.quotas From 65650ba4dcba8a9729cb9cb6477a62a7b7c3714e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 9 Mar 2017 09:27:33 -0800 Subject: [PATCH 083/101] MINOR: Introduce NetworkClient.hasInFlightRequests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It’s a minor optimisation, but simple enough. Author: Ismael Juma Reviewers: Apurva Mehta , Jason Gustafson Closes #2658 from ijuma/has-in-flight-requests --- .../kafka/clients/InFlightRequests.java | 27 ++++++++++++++++--- .../org/apache/kafka/clients/KafkaClient.java | 16 ++++++++--- .../apache/kafka/clients/NetworkClient.java | 16 ++++++++--- .../internals/ConsumerNetworkClient.java | 4 +-- .../org/apache/kafka/clients/MockClient.java | 17 +++++++++++- .../kafka/clients/NetworkClientTest.java | 3 +++ .../producer/internals/SenderTest.java | 11 +++++++- 7 files changed, 80 insertions(+), 14 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java index 06b8fedad1348..a29075df4df4b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java +++ b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java @@ -55,7 +55,7 @@ public void add(NetworkClient.InFlightRequest request) { private Deque requestQueue(String node) { Deque reqs = requests.get(node); if (reqs == null || reqs.isEmpty()) - throw new IllegalStateException("Response from server for which there are no in-flight requests."); + throw new IllegalStateException("There are no in-flight requests for node " + node); return reqs; } @@ -96,25 +96,44 @@ public boolean canSendMore(String node) { } /** - * Return the number of inflight requests directed at the given node + * Return the number of in-flight requests directed at the given node * @param node The node * @return The request count. */ - public int inFlightRequestCount(String node) { + public int count(String node) { Deque queue = requests.get(node); return queue == null ? 0 : queue.size(); } + /** + * Return true if there is no in-flight request directed at the given node and false otherwise + */ + public boolean isEmpty(String node) { + Deque queue = requests.get(node); + return queue != null && !queue.isEmpty(); + } + /** * Count all in-flight requests for all nodes */ - public int inFlightRequestCount() { + public int count() { int total = 0; for (Deque deque : this.requests.values()) total += deque.size(); return total; } + /** + * Return true if there is no in-flight request and false otherwise + */ + public boolean isEmpty() { + for (Deque deque : this.requests.values()) { + if (!deque.isEmpty()) + return false; + } + return false; + } + /** * Clear out all the in-flight requests for the given node and return them * diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 86ffa498bdbaf..83a0009113d78 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -107,6 +107,11 @@ public interface KafkaClient extends Closeable { */ int inFlightRequestCount(); + /** + * Return true if there is at least one in-flight request and false otherwise. + */ + boolean hasInFlightRequests(); + /** * Get the total in-flight requests for a particular node * @@ -114,6 +119,11 @@ public interface KafkaClient extends Closeable { */ int inFlightRequestCount(String nodeId); + /** + * Return true if there is at least one in-flight request for a particular node and false otherwise. + */ + boolean hasInFlightRequests(String nodeId); + /** * Wake up the client if it is currently blocked waiting for I/O */ @@ -139,7 +149,7 @@ ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder request * @param expectResponse true iff we expect a response * @param callback the callback to invoke when we get a response */ - ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder requestBuilder, - long createdTimeMs, boolean expectResponse, - RequestCompletionHandler callback); + ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder requestBuilder, long createdTimeMs, + boolean expectResponse, RequestCompletionHandler callback); + } diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index c76a738f071c6..b6f8b0ed4fc0b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -386,7 +386,12 @@ public List poll(long timeout, long now) { */ @Override public int inFlightRequestCount() { - return this.inFlightRequests.inFlightRequestCount(); + return this.inFlightRequests.count(); + } + + @Override + public boolean hasInFlightRequests() { + return !this.inFlightRequests.isEmpty(); } /** @@ -394,7 +399,12 @@ public int inFlightRequestCount() { */ @Override public int inFlightRequestCount(String node) { - return this.inFlightRequests.inFlightRequestCount(node); + return this.inFlightRequests.count(node); + } + + @Override + public boolean hasInFlightRequests(String node) { + return this.inFlightRequests.isEmpty(node); } /** @@ -431,7 +441,7 @@ public Node leastLoadedNode(long now) { for (int i = 0; i < nodes.size(); i++) { int idx = (offset + i) % nodes.size(); Node node = nodes.get(idx); - int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString()); + int currInflight = this.inFlightRequests.count(node.idString()); if (currInflight == 0 && this.connectionStates.isReady(node.idString())) { // if we find an established connection with no in-flight requests we can stop right away log.trace("Found least loaded node {} connected with no in-flight requests", node); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 478ed3f7d0024..890fe7ad9d480 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -304,7 +304,7 @@ public boolean hasPendingRequests(Node node) { if (unsent.hasRequests(node)) return true; synchronized (this) { - return client.inFlightRequestCount(node.idString()) > 0; + return client.hasInFlightRequests(node.idString()); } } @@ -328,7 +328,7 @@ public boolean hasPendingRequests() { if (unsent.hasRequests()) return true; synchronized (this) { - return client.inFlightRequestCount() > 0; + return client.hasInFlightRequests(); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 7e05881243350..3726f1adbec17 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -295,9 +295,24 @@ public int inFlightRequestCount() { return requests.size(); } + @Override + public boolean hasInFlightRequests() { + return !requests.isEmpty(); + } + @Override public int inFlightRequestCount(String node) { - return requests.size(); + int result = 0; + for (ClientRequest req : requests) { + if (req.destination().equals(node)) + ++result; + } + return result; + } + + @Override + public boolean hasInFlightRequests(String node) { + return inFlightRequestCount(node) > 0; } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index d22c04afe263d..de60f442d4531 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -116,9 +116,11 @@ public void testClose() { client.send(request, time.milliseconds()); assertEquals("There should be 1 in-flight request after send", 1, client.inFlightRequestCount(node.idString())); + assertTrue(client.hasInFlightRequests(node.idString())); client.close(node.idString()); assertEquals("There should be no in-flight request after close", 0, client.inFlightRequestCount(node.idString())); + assertFalse(client.hasInFlightRequests(node.idString())); assertFalse("Connection should not be ready after close", client.isReady(node, 0)); } @@ -247,6 +249,7 @@ public void testDisconnectDuringUserMetadataRequest() { client.send(request, now); client.poll(requestTimeoutMs, now); assertEquals(1, client.inFlightRequestCount(node.idString())); + assertTrue(client.hasInFlightRequests(node.idString())); selector.close(node.idString()); List responses = client.poll(requestTimeoutMs, time.milliseconds()); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 50ea219015c36..8a8079065e72c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -103,9 +103,11 @@ public void testSimple() throws Exception { sender.run(time.milliseconds()); // connect sender.run(time.milliseconds()); // send produce request assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount()); + assertTrue(client.hasInFlightRequests()); client.respond(produceResponse(tp, offset, Errors.NONE, 0)); sender.run(time.milliseconds()); - assertEquals("All requests completed.", offset, (long) client.inFlightRequestCount()); + assertEquals("All requests completed.", 0, client.inFlightRequestCount()); + assertFalse(client.hasInFlightRequests()); sender.run(time.milliseconds()); assertTrue("Request should be completed", future.isDone()); assertEquals(offset, future.get().offset()); @@ -153,14 +155,17 @@ public void testRetries() throws Exception { String id = client.requests().peek().destination(); Node node = new Node(Integer.parseInt(id), "localhost", 0); assertEquals(1, client.inFlightRequestCount()); + assertTrue(client.hasInFlightRequests()); assertTrue("Client ready status should be true", client.isReady(node, 0L)); client.disconnect(id); assertEquals(0, client.inFlightRequestCount()); + assertFalse(client.hasInFlightRequests()); assertFalse("Client ready status should be false", client.isReady(node, 0L)); sender.run(time.milliseconds()); // receive error sender.run(time.milliseconds()); // reconnect sender.run(time.milliseconds()); // resend assertEquals(1, client.inFlightRequestCount()); + assertTrue(client.hasInFlightRequests()); long offset = 0; client.respond(produceResponse(tp, offset, Errors.NONE, 0)); sender.run(time.milliseconds()); @@ -212,6 +217,7 @@ public void testSendInOrder() throws Exception { assertEquals(ApiKeys.PRODUCE, client.requests().peek().requestBuilder().apiKey()); Node node = new Node(Integer.parseInt(id), "localhost", 0); assertEquals(1, client.inFlightRequestCount()); + assertTrue(client.hasInFlightRequests()); assertTrue("Client ready status should be true", client.isReady(node, 0L)); time.sleep(900); @@ -224,6 +230,7 @@ public void testSendInOrder() throws Exception { // Sender should not send the second message to node 0. sender.run(time.milliseconds()); assertEquals(1, client.inFlightRequestCount()); + assertTrue(client.hasInFlightRequests()); } finally { m.close(); } @@ -246,6 +253,7 @@ public void testMetadataTopicExpiry() throws Exception { client.respond(produceResponse(tp, offset++, Errors.NONE, 0)); sender.run(time.milliseconds()); assertEquals("Request completed.", 0, client.inFlightRequestCount()); + assertFalse(client.hasInFlightRequests()); sender.run(time.milliseconds()); assertTrue("Request should be completed", future.isDone()); @@ -261,6 +269,7 @@ public void testMetadataTopicExpiry() throws Exception { client.respond(produceResponse(tp, offset++, Errors.NONE, 0)); sender.run(time.milliseconds()); assertEquals("Request completed.", 0, client.inFlightRequestCount()); + assertFalse(client.hasInFlightRequests()); sender.run(time.milliseconds()); assertTrue("Request should be completed", future.isDone()); } From dbcbd7920f7056c80d55453ad543ea9ba87da9be Mon Sep 17 00:00:00 2001 From: Raghav Kumar Gautam Date: Thu, 9 Mar 2017 16:24:38 -0800 Subject: [PATCH 084/101] KAFKA-4467: Run tests on travis-ci using docker ijuma ewencp cmccabe harshach Please review. Here is a sample run: https://travis-ci.org/raghavgautam/kafka/builds/191714520 In this run 214 tests were run and 144 tests passed. I will open separate jiras for fixing failures. Author: Raghav Kumar Gautam Reviewers: Sriharsha Chintalapani , Ewen Cheslack-Postava Closes #2376 from raghavgautam/trunk --- .travis.yml | 54 +++++++ tests/README.md | 303 ++++++++++++++++++++++++++++++++++++++++ tests/docker/Dockerfile | 2 +- 3 files changed, 358 insertions(+), 1 deletion(-) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 0000000000000..9be5c58d9e9ac --- /dev/null +++ b/.travis.yml @@ -0,0 +1,54 @@ +# Licensed 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. + +sudo: required +dist: trusty +language: java + +env: + - _DUCKTAPE_OPTIONS="--subset 0 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 1 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 2 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 3 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 4 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 5 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 6 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 7 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 8 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 9 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 10 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 11 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 12 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 13 --subsets 15" + - _DUCKTAPE_OPTIONS="--subset 14 --subsets 15" + +jdk: + - oraclejdk8 + +before_install: + - gradle wrapper + +script: + - ./gradlew rat + - ./gradlew releaseTarGz && /bin/bash ./tests/docker/run_tests.sh + +services: + - docker + +before_cache: + - rm -f $HOME/.gradle/caches/modules-2/modules-2.lock + - rm -fr $HOME/.gradle/caches/*/plugin-resolution/ +cache: + directories: + - "$HOME/.m2/repository" + - "$HOME/.gradle/caches/" + - "$HOME/.gradle/wrapper/" diff --git a/tests/README.md b/tests/README.md index 87a6fed446ae1..ec6ab31a79257 100644 --- a/tests/README.md +++ b/tests/README.md @@ -31,6 +31,309 @@ TC_PATHS="tests/kafkatest/tests/streams tests/kafkatest/tests/tools" bash tests/ - The docker containers are named knode01, knode02 etc. These nodes can't be used for any other purpose. +Examining CI run +---------------- +* Set BUILD_ID is travis ci's build id. E.g. build id is 169519874 for the following build +```bash +https://travis-ci.org/apache/kafka/builds/169519874 +``` + +* Getting number of tests that were actually run +```bash +for id in $(curl -sSL https://api.travis-ci.org/builds/$BUILD_ID | jq '.matrix|map(.id)|.[]'); do curl -sSL "https://api.travis-ci.org/jobs/$id/log.txt?deansi=true" ; done | grep -cE 'RunnerClient: Loading test' +``` + +* Getting number of tests that passed +```bash +for id in $(curl -sSL https://api.travis-ci.org/builds/$BUILD_ID | jq '.matrix|map(.id)|.[]'); do curl -sSL "https://api.travis-ci.org/jobs/$id/log.txt?deansi=true" ; done | grep -cE 'RunnerClient.*PASS' +``` +* Getting all the logs produced from a run +```bash +for id in $(curl -sSL https://api.travis-ci.org/builds/$BUILD_ID | jq '.matrix|map(.id)|.[]'); do curl -sSL "https://api.travis-ci.org/jobs/$id/log.txt?deansi=true" ; done +``` +* Explanation of curl calls to travis-ci & jq commands + - We get json information of the build using the following command +```bash +curl -sSL https://api.travis-ci.org/apache/kafka/builds/169519874 +``` +This produces a json about the build which looks like: +```json +{ + "id": 169519874, + "repository_id": 6097916, + "number": "19", + "config": { + "sudo": "required", + "dist": "trusty", + "language": "java", + "env": [ + "TC_PATHS=\"tests/kafkatest/tests/client\"", + "TC_PATHS=\"tests/kafkatest/tests/connect tests/kafkatest/tests/streams tests/kafkatest/tests/tools\"", + "TC_PATHS=\"tests/kafkatest/tests/mirror_maker\"", + "TC_PATHS=\"tests/kafkatest/tests/replication\"", + "TC_PATHS=\"tests/kafkatest/tests/upgrade\"", + "TC_PATHS=\"tests/kafkatest/tests/security\"", + "TC_PATHS=\"tests/kafkatest/tests/core\"" + ], + "jdk": [ + "oraclejdk8" + ], + "before_install": null, + "script": [ + "./gradlew releaseTarGz && /bin/bash ./tests/travis/run_tests.sh" + ], + "services": [ + "docker" + ], + "before_cache": [ + "rm -f $HOME/.gradle/caches/modules-2/modules-2.lock", + "rm -fr $HOME/.gradle/caches/*/plugin-resolution/" + ], + "cache": { + "directories": [ + "$HOME/.m2/repository", + "$HOME/.gradle/caches/", + "$HOME/.gradle/wrapper/" + ] + }, + ".result": "configured", + "group": "stable" + }, + "state": "finished", + "result": null, + "status": null, + "started_at": "2016-10-21T13:35:43Z", + "finished_at": "2016-10-21T14:46:03Z", + "duration": 16514, + "commit": "7e583d9ea08c70dbbe35a3adde72ed203a797f64", + "branch": "trunk", + "message": "respect _DUCK_OPTIONS", + "committed_at": "2016-10-21T00:12:36Z", + "author_name": "Raghav Kumar Gautam", + "author_email": "raghav@apache.org", + "committer_name": "Raghav Kumar Gautam", + "committer_email": "raghav@apache.org", + "compare_url": "https://github.com/raghavgautam/kafka/compare/cc788ac99ca7...7e583d9ea08c", + "event_type": "push", + "matrix": [ + { + "id": 169519875, + "repository_id": 6097916, + "number": "19.1", + "config": { + "sudo": "required", + "dist": "trusty", + "language": "java", + "env": "TC_PATHS=\"tests/kafkatest/tests/client\"", + "jdk": "oraclejdk8", + "before_install": null, + "script": [ + "./gradlew releaseTarGz && /bin/bash ./tests/travis/run_tests.sh" + ], + "services": [ + "docker" + ], + "before_cache": [ + "rm -f $HOME/.gradle/caches/modules-2/modules-2.lock", + "rm -fr $HOME/.gradle/caches/*/plugin-resolution/" + ], + "cache": { + "directories": [ + "$HOME/.m2/repository", + "$HOME/.gradle/caches/", + "$HOME/.gradle/wrapper/" + ] + }, + ".result": "configured", + "group": "stable", + "os": "linux" + }, + "result": null, + "started_at": "2016-10-21T13:35:43Z", + "finished_at": "2016-10-21T14:24:50Z", + "allow_failure": false + }, + { + "id": 169519876, + "repository_id": 6097916, + "number": "19.2", + "config": { + "sudo": "required", + "dist": "trusty", + "language": "java", + "env": "TC_PATHS=\"tests/kafkatest/tests/connect tests/kafkatest/tests/streams tests/kafkatest/tests/tools\"", + "jdk": "oraclejdk8", + "before_install": null, + "script": [ + "./gradlew releaseTarGz && /bin/bash ./tests/travis/run_tests.sh" + ], + "services": [ + "docker" + ], + "before_cache": [ + "rm -f $HOME/.gradle/caches/modules-2/modules-2.lock", + "rm -fr $HOME/.gradle/caches/*/plugin-resolution/" + ], + "cache": { + "directories": [ + "$HOME/.m2/repository", + "$HOME/.gradle/caches/", + "$HOME/.gradle/wrapper/" + ] + }, + ".result": "configured", + "group": "stable", + "os": "linux" + }, + "result": 1, + "started_at": "2016-10-21T13:35:46Z", + "finished_at": "2016-10-21T14:22:05Z", + "allow_failure": false + }, + + ... + ] +} + +``` + - By passing this through jq filter `.matrix` we extract the matrix part of the json +```bash +curl -sSL https://api.travis-ci.org/apache/kafka/builds/169519874 | jq '.matrix' +``` +The resulting json looks like: +```json +[ + { + "id": 169519875, + "repository_id": 6097916, + "number": "19.1", + "config": { + "sudo": "required", + "dist": "trusty", + "language": "java", + "env": "TC_PATHS=\"tests/kafkatest/tests/client\"", + "jdk": "oraclejdk8", + "before_install": null, + "script": [ + "./gradlew releaseTarGz && /bin/bash ./tests/travis/run_tests.sh" + ], + "services": [ + "docker" + ], + "before_cache": [ + "rm -f $HOME/.gradle/caches/modules-2/modules-2.lock", + "rm -fr $HOME/.gradle/caches/*/plugin-resolution/" + ], + "cache": { + "directories": [ + "$HOME/.m2/repository", + "$HOME/.gradle/caches/", + "$HOME/.gradle/wrapper/" + ] + }, + ".result": "configured", + "group": "stable", + "os": "linux" + }, + "result": null, + "started_at": "2016-10-21T13:35:43Z", + "finished_at": "2016-10-21T14:24:50Z", + "allow_failure": false + }, + { + "id": 169519876, + "repository_id": 6097916, + "number": "19.2", + "config": { + "sudo": "required", + "dist": "trusty", + "language": "java", + "env": "TC_PATHS=\"tests/kafkatest/tests/connect tests/kafkatest/tests/streams tests/kafkatest/tests/tools\"", + "jdk": "oraclejdk8", + "before_install": null, + "script": [ + "./gradlew releaseTarGz && /bin/bash ./tests/travis/run_tests.sh" + ], + "services": [ + "docker" + ], + "before_cache": [ + "rm -f $HOME/.gradle/caches/modules-2/modules-2.lock", + "rm -fr $HOME/.gradle/caches/*/plugin-resolution/" + ], + "cache": { + "directories": [ + "$HOME/.m2/repository", + "$HOME/.gradle/caches/", + "$HOME/.gradle/wrapper/" + ] + }, + ".result": "configured", + "group": "stable", + "os": "linux" + }, + "result": 1, + "started_at": "2016-10-21T13:35:46Z", + "finished_at": "2016-10-21T14:22:05Z", + "allow_failure": false + }, + + ... +] + +``` + - By further passing this through jq filter `map(.id)` we extract the id of + the builds for each of the splits +```bash +curl -sSL https://api.travis-ci.org/apache/kafka/builds/169519874 | jq '.matrix|map(.id)' +``` +The resulting json looks like: +```json +[ + 169519875, + 169519876, + 169519877, + 169519878, + 169519879, + 169519880, + 169519881 +] +``` + - To use these ids in for loop we want to get rid of `[]` which is done by + passing it through `.[]` filter +```bash +curl -sSL https://api.travis-ci.org/apache/kafka/builds/169519874 | jq '.matrix|map(.id)|.[]' +``` +And we get +```text +169519875 +169519876 +169519877 +169519878 +169519879 +169519880 +169519881 +``` + - In the for loop we have made calls to fetch logs +```bash +curl -sSL "https://api.travis-ci.org/jobs/169519875/log.txt?deansi=true" | tail +``` +which gives us +```text +[INFO:2016-10-21 14:21:12,538]: SerialTestRunner: kafkatest.tests.client.consumer_test.OffsetValidationTest.test_consumer_bounce.clean_shutdown=False.bounce_mode=rolling: test 16 of 28 +[INFO:2016-10-21 14:21:12,538]: SerialTestRunner: kafkatest.tests.client.consumer_test.OffsetValidationTest.test_consumer_bounce.clean_shutdown=False.bounce_mode=rolling: setting up +[INFO:2016-10-21 14:21:30,810]: SerialTestRunner: kafkatest.tests.client.consumer_test.OffsetValidationTest.test_consumer_bounce.clean_shutdown=False.bounce_mode=rolling: running +[INFO:2016-10-21 14:24:35,519]: SerialTestRunner: kafkatest.tests.client.consumer_test.OffsetValidationTest.test_consumer_bounce.clean_shutdown=False.bounce_mode=rolling: PASS +[INFO:2016-10-21 14:24:35,519]: SerialTestRunner: kafkatest.tests.client.consumer_test.OffsetValidationTest.test_consumer_bounce.clean_shutdown=False.bounce_mode=rolling: tearing down + + +The job exceeded the maximum time limit for jobs, and has been terminated. + +``` +* Links + - [Travis-CI REST api documentation](https://docs.travis-ci.com/api) + - [jq Manual](https://stedolan.github.io/jq/manual/) + Local Quickstart ---------------- This quickstart will help you run the Kafka system tests on your local machine. Note this requires bringing up a cluster of virtual machines on your local computer, which is memory intensive; it currently requires around 10G RAM. diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 78b2e32d35278..b349f7edae727 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -21,7 +21,7 @@ ENV MIRROR="http://apache.cs.utah.edu/" ADD ssh /root/.ssh RUN chmod 600 /root/.ssh/id_rsa -RUN apt update && apt install -y unzip wget curl jq coreutils openssh-server net-tools vim openjdk-8-jdk python-pip python-dev libffi-dev libssl-dev +RUN apt update && apt install -y unzip wget curl jq coreutils openssh-server net-tools vim python-pip python-dev libffi-dev libssl-dev RUN pip install -U pip && pip install --upgrade cffi ducktape==0.6.0 RUN mkdir -p "/opt/kafka-0.8.2.2" && curl -s "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.8.2.2" RUN mkdir -p "/opt/kafka-0.9.0.1" && curl -s "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.9.0.1" From 7565dcd8b0547f91a5d9d19771d9cd6693079d01 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 9 Mar 2017 16:45:41 -0800 Subject: [PATCH 085/101] KAFKA-4861; GroupMetadataManager record is rejected if broker configured with LogAppendTime The record should be created with CreateTime (like in the producer). The conversion to LogAppendTime happens automatically (if necessary). Author: Ismael Juma Reviewers: Jason Gustafson Closes #2657 from ijuma/kafka-4861-log-append-time-breaks-group-data-manager --- .../coordinator/GroupMetadataManager.scala | 33 ++++--- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 6 +- .../kafka/api/BaseConsumerTest.scala | 2 +- .../kafka/api/ConsumerBounceTest.scala | 3 +- .../kafka/api/FixedPortTestUtils.scala | 9 +- .../kafka/api/IntegrationTestHarness.scala | 2 +- .../kafka/api/LogAppendTimeTest.scala | 86 +++++++++++++++++++ .../GroupCoordinatorResponseTest.scala | 12 +-- .../GroupMetadataManagerTest.scala | 16 ++-- .../integration/KafkaServerTestHarness.scala | 4 +- 11 files changed, 128 insertions(+), 47 deletions(-) create mode 100644 core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index a6ed6a9aad98e..d48328db073c0 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -133,8 +133,8 @@ class GroupMetadataManager(val brokerId: Int, def prepareStoreGroup(group: GroupMetadata, groupAssignment: Map[String, Array[Byte]], responseCallback: Errors => Unit): Option[DelayedStore] = { - getMagicAndTimestamp(partitionFor(group.groupId)) match { - case Some((magicValue, timestampType, timestamp)) => + getMagic(partitionFor(group.groupId)) match { + case Some(magicValue) => val groupMetadataValueVersion = { if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0) 0.toShort @@ -142,6 +142,9 @@ class GroupMetadataManager(val brokerId: Int, GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION } + // We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + val timestampType = TimestampType.CREATE_TIME + val timestamp = time.milliseconds() val record = Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.groupMetadataKey(group.groupId), GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion)) @@ -231,8 +234,11 @@ class GroupMetadataManager(val brokerId: Int, } // construct the message set to append - getMagicAndTimestamp(partitionFor(group.groupId)) match { - case Some((magicValue, timestampType, timestamp)) => + getMagic(partitionFor(group.groupId)) match { + case Some(magicValue) => + // We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + val timestampType = TimestampType.CREATE_TIME + val timestamp = time.milliseconds() val records = filteredOffsetMetadata.map { case (topicPartition, offsetAndMetadata) => Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.offsetCommitKey(group.groupId, topicPartition), @@ -575,8 +581,12 @@ class GroupMetadataManager(val brokerId: Int, val offsetsPartition = partitionFor(groupId) val appendPartition = new TopicPartition(Topic.GroupMetadataTopicName, offsetsPartition) - getMagicAndTimestamp(offsetsPartition) match { - case Some((magicValue, timestampType, timestamp)) => + getMagic(offsetsPartition) match { + case Some(magicValue) => + // We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + val timestampType = TimestampType.CREATE_TIME + val timestamp = time.milliseconds() + val partitionOpt = replicaManager.getPartition(appendPartition) partitionOpt.foreach { partition => val tombstones = removedOffsets.map { case (topicPartition, offsetAndMetadata) => @@ -652,15 +662,10 @@ class GroupMetadataManager(val brokerId: Int, * Check if the replica is local and return the message format version and timestamp * * @param partition Partition of GroupMetadataTopic - * @return Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise + * @return Some(MessageFormatVersion) if replica is local, None otherwise */ - private def getMagicAndTimestamp(partition: Int): Option[(Byte, TimestampType, Long)] = { - val groupMetadataTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partition) - replicaManager.getMagicAndTimestampType(groupMetadataTopicPartition).map { case (messageFormatVersion, timestampType) => - val timestamp = if (messageFormatVersion == Record.MAGIC_VALUE_V0) Record.NO_TIMESTAMP else time.milliseconds() - (messageFormatVersion, timestampType, timestamp) - } - } + private def getMagic(partition: Int): Option[Byte] = + replicaManager.getMagic(new TopicPartition(Topic.GroupMetadataTopicName, partition)) /** * Add the partition into the owned list diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index fa5afe5f68ba4..24a224a1fe758 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -470,7 +470,7 @@ class KafkaApis(val requestChannel: RequestChannel, // Please note that if the message format is changed from a higher version back to lower version this // test might break because some messages in new message format can be delivered to consumers before 0.10.0.0 // without format down conversion. - val convertedData = if (versionId <= 1 && replicaManager.getMagicAndTimestampType(tp).exists(_._1 > Record.MAGIC_VALUE_V0) && + val convertedData = if (versionId <= 1 && replicaManager.getMagic(tp).exists(_ > Record.MAGIC_VALUE_V0) && !data.records.hasMatchingShallowMagic(Record.MAGIC_VALUE_V0)) { trace(s"Down converting message to V0 for fetch request from $clientId") val downConvertedRecords = data.records.toMessageFormat(Record.MAGIC_VALUE_V0, TimestampType.NO_TIMESTAMP_TYPE) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 1cec4a22a3d1c..4ab8c2a2928f2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -618,10 +618,8 @@ class ReplicaManager(val config: KafkaConfig, quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync } - def getMagicAndTimestampType(topicPartition: TopicPartition): Option[(Byte, TimestampType)] = - getReplica(topicPartition).flatMap { replica => - replica.log.map(log => (log.config.messageFormatVersion.messageFormatVersion, log.config.messageTimestampType)) - } + def getMagic(topicPartition: TopicPartition): Option[Byte] = + getReplica(topicPartition).flatMap(_.log.map(_.config.messageFormatVersion.messageFormatVersion)) def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) : Seq[TopicPartition] = { replicaStateChangeLock synchronized { diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 802bab89233f6..27b89d5d1faf0 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.errors.WakeupException /** * Integration tests for the new consumer that cover basic usage as well as server failures */ -abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { +abstract class BaseConsumerTest extends IntegrationTestHarness { val epsilon = 0.1 val producerCount = 1 diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index f0e0c9e9bac0d..4ec77a153c8ba 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -46,7 +46,6 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { val executor = Executors.newScheduledThreadPool(2) // configure the servers and clients - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") this.serverConfig.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "10") // set small enough session timeout @@ -59,7 +58,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") override def generateConfigs() = { - FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect,enableControlledShutdown = false) + FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect, enableControlledShutdown = false) .map(KafkaConfig.fromProps(_, serverConfig)) } diff --git a/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala index d15a01d055c21..bf5f8c1e6a3ec 100644 --- a/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala +++ b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala @@ -34,9 +34,7 @@ object FixedPortTestUtils { sockets.foreach(_.close()) ports } catch { - case e: IOException => { - throw new RuntimeException(e) - } + case e: IOException => throw new RuntimeException(e) } } @@ -45,8 +43,9 @@ object FixedPortTestUtils { enableControlledShutdown: Boolean = true, enableDeleteTopic: Boolean = false): Seq[Properties] = { val ports = FixedPortTestUtils.choosePorts(numConfigs) - (0 until numConfigs) - .map(node => TestUtils.createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, ports(node))) + (0 until numConfigs).map { node => + TestUtils.createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, ports(node)) + } } } diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 46465e8f28cd9..5c8ceeacdb6e3 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -45,7 +45,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() - override def generateConfigs() = { + override def generateConfigs = { val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol), trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties) cfgs.foreach { config => diff --git a/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala new file mode 100644 index 0000000000000..4a97bea680b26 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala @@ -0,0 +1,86 @@ +/* + * 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.util.Collections +import java.util.concurrent.TimeUnit + +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.record.TimestampType +import org.junit.{Before, Test} +import org.junit.Assert.{assertEquals, assertNotEquals, assertTrue} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +/** + * Tests where the broker is configured to use LogAppendTime. For tests where LogAppendTime is configured via topic + * level configs, see the *ProducerSendTest classes. + */ +class LogAppendTimeTest extends IntegrationTestHarness { + val producerCount: Int = 1 + val consumerCount: Int = 1 + val serverCount: Int = 2 + + // This will be used for the offsets topic as well + serverConfig.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.name) + serverConfig.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "2") + + private val topic = "topic" + + @Before + override def setUp() { + super.setUp() + TestUtils.createTopic(zkUtils, topic, servers = servers) + } + + @Test + def testProduceConsume() { + val producer = producers.head + val now = System.currentTimeMillis() + val createTime = now - TimeUnit.DAYS.toMillis(1) + val producerRecords = (1 to 10).map(i => new ProducerRecord(topic, null, createTime, s"key$i".getBytes, + s"value$i".getBytes)) + val recordMetadatas = producerRecords.map(producer.send).map(_.get(10, TimeUnit.SECONDS)) + recordMetadatas.foreach { recordMetadata => + assertTrue(recordMetadata.timestamp >= now) + assertTrue(recordMetadata.timestamp < now + TimeUnit.SECONDS.toMillis(60)) + } + + val consumer = consumers.head + consumer.subscribe(Collections.singleton(topic)) + val consumerRecords = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] + TestUtils.waitUntilTrue(() => { + consumerRecords ++= consumer.poll(50).asScala + consumerRecords.size == producerRecords.size + }, s"Consumed ${consumerRecords.size} records until timeout instead of the expected ${producerRecords.size} records") + + consumerRecords.zipWithIndex.foreach { case (consumerRecord, index) => + val producerRecord = producerRecords(index) + val recordMetadata = recordMetadatas(index) + assertEquals(new String(producerRecord.key), new String(consumerRecord.key)) + assertEquals(new String(producerRecord.value), new String(consumerRecord.value)) + assertNotEquals(producerRecord.timestamp, consumerRecord.timestamp) + assertEquals(recordMetadata.timestamp, consumerRecord.timestamp) + assertEquals(TimestampType.LOG_APPEND_TIME, consumerRecord.timestampType) + } + } + +} diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 7b401875e9855..22cb899d004ad 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -305,8 +305,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.reset(replicaManager) EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andReturn(None) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) timer.advanceClock(DefaultSessionTimeout + 100) @@ -1055,8 +1054,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { new PartitionResponse(Errors.NONE, 0L, Record.NO_TIMESTAMP) ) )}) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) @@ -1137,8 +1135,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { new PartitionResponse(Errors.NONE, 0L, Record.NO_TIMESTAMP) ) )}) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback) @@ -1149,8 +1146,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val (responseFuture, responseCallback) = setupHeartbeatCallback EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andReturn(None) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala index 86189aa0cf186..8cfae8d76b6b2 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala @@ -321,7 +321,7 @@ class GroupMetadataManagerTest { @Test def testStoreNonEmptyGroupWhenCoordinatorHasMoved() { - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(None) val memberId = "memberId" val clientId = "clientId" val clientHost = "localhost" @@ -390,7 +390,7 @@ class GroupMetadataManagerTest { @Test def testCommitOffsetWhenCoordinatorHasMoved() { - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(None) val memberId = "" val generationId = -1 val topicPartition = new TopicPartition("foo", 0) @@ -538,8 +538,7 @@ class GroupMetadataManagerTest { EasyMock.reset(partition) val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture() - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(Record.MAGIC_VALUE_V1)) EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andStubReturn(Some(partition)) EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt())) .andReturn(LogAppendInfo.UnknownLogAppendInfo) @@ -584,8 +583,7 @@ class GroupMetadataManagerTest { EasyMock.reset(partition) val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture() - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.LOG_APPEND_TIME)) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(Record.MAGIC_VALUE_V1)) EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andStubReturn(Some(partition)) EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt())) .andReturn(LogAppendInfo.UnknownLogAppendInfo) @@ -602,7 +600,8 @@ class GroupMetadataManagerTest { assertTrue(metadataTombstone.hasKey) assertTrue(metadataTombstone.hasNullValue) assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic) - assertEquals(TimestampType.LOG_APPEND_TIME, metadataTombstone.timestampType) + // Use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + assertEquals(TimestampType.CREATE_TIME, metadataTombstone.timestampType) assertTrue(metadataTombstone.timestamp > 0) val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey] @@ -762,8 +761,7 @@ class GroupMetadataManagerTest { new PartitionResponse(error, 0L, Record.NO_TIMESTAMP) ) )}) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(Record.MAGIC_VALUE_V1)) } private def buildStableGroupRecordWithMember(memberId: String): Record = { diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 6fa7ad5147812..9f40ec6bae57c 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -47,7 +47,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every * test and should not reuse previous configurations unless they select their ports randomly when servers are started. */ - def generateConfigs(): Seq[KafkaConfig] + def generateConfigs: Seq[KafkaConfig] /** * Override this in case ACLs or security credentials must be set before `servers` are started. @@ -64,7 +64,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { def configs: Seq[KafkaConfig] = { if (instanceConfigs == null) - instanceConfigs = generateConfigs() + instanceConfigs = generateConfigs instanceConfigs } From c063561ce815ec066851e480b7f70a84d6ccd371 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Fri, 10 Mar 2017 07:51:42 -0800 Subject: [PATCH 086/101] MINOR: KStream: fix typo in javadoc Author: Michael G. Noll Reviewers: Guozhang Wang, Damian Guy Closes #2668 from miguno/trunk-kstream-javadoc-typo --- .../main/java/org/apache/kafka/streams/kstream/KStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index a11d8f443b8a7..f41218d2004f3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -1536,7 +1536,7 @@ KStream join(final KTable table, * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and * will not produce any result records. *

    - * For each {@code KStream} record weather or not it finds a corresponding record in {@link KTable} the provided + * For each {@code KStream} record whether or not it finds a corresponding record in {@link KTable} the provided * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoiner}. * The key of the result record is the same as for both joining input records. @@ -1608,7 +1608,7 @@ KStream leftJoin(final KTable table, * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and * will not produce any result records. *

    - * For each {@code KStream} record weather or not it finds a corresponding record in {@link KTable} the provided + * For each {@code KStream} record whether or not it finds a corresponding record in {@link KTable} the provided * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoiner}. * The key of the result record is the same as for both joining input records. From dfa2ef4837259cd127b013e63ce26acdec8b94c5 Mon Sep 17 00:00:00 2001 From: "sharad.develop" Date: Fri, 10 Mar 2017 11:28:06 -0800 Subject: [PATCH 087/101] KAFKA-4738: Remove generic type of class ClientState Remove generic type of class ClientState and generic T inTaskAssignor. Author: sharad.develop Reviewers: Matthias J. Sax, Guozhang Wang Closes #2616 from sharad-develop/KAFKA-4738 --- .../internals/StreamPartitionAssignor.java | 10 +- .../internals/assignment/ClientState.java | 60 ++++++------ .../assignment/StickyTaskAssignor.java | 50 +++++----- .../internals/assignment/ClientStateTest.java | 92 +++++++++++-------- .../assignment/StickyTaskAssignorTest.java | 56 +++++------ 5 files changed, 143 insertions(+), 125 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index e3f66984adb7f..859d66112282e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -79,7 +79,7 @@ public int compareTo(final AssignedPartition that) { private static class ClientMetadata { final HostInfo hostInfo; final Set consumers; - final ClientState state; + final ClientState state; ClientMetadata(final String endPoint) { @@ -100,7 +100,7 @@ private static class ClientMetadata { consumers = new HashSet<>(); // initialize the client state - state = new ClientState<>(); + state = new ClientState(); } void addConsumer(final String consumerMemberId, final SubscriptionInfo info) { @@ -449,7 +449,7 @@ public Map assign(Cluster metadata, Map> states = new HashMap<>(); + Map states = new HashMap<>(); for (Map.Entry entry : clientsMetadata.entrySet()) { states.put(entry.getKey(), entry.getValue().state); } @@ -471,7 +471,7 @@ public Map assign(Cluster metadata, Map topicPartitions = new HashSet<>(); - final ClientState state = entry.getValue().state; + final ClientState state = entry.getValue().state; for (final TaskId id : state.activeTasks()) { topicPartitions.addAll(partitionsForTask.get(id)); @@ -485,7 +485,7 @@ public Map assign(Cluster metadata, Map assignment = new HashMap<>(); for (Map.Entry entry : clientsMetadata.entrySet()) { final Set consumers = entry.getValue().consumers; - final ClientState state = entry.getValue().state; + final ClientState state = entry.getValue().state; final ArrayList taskIds = new ArrayList<>(state.assignedTaskCount()); final int numActiveTasks = state.activeTaskCount(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java index 99bd29ee7956b..15ee849bffc33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java @@ -16,28 +16,30 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import org.apache.kafka.streams.processor.TaskId; import java.util.HashSet; import java.util.Set; -public class ClientState { - private final Set activeTasks; - private final Set standbyTasks; - private final Set assignedTasks; - private final Set prevActiveTasks; - private final Set prevAssignedTasks; +public class ClientState { + private final Set activeTasks; + private final Set standbyTasks; + private final Set assignedTasks; + private final Set prevActiveTasks; + private final Set prevAssignedTasks; private int capacity; + public ClientState() { this(0); } ClientState(final int capacity) { - this(new HashSet(), new HashSet(), new HashSet(), new HashSet(), new HashSet(), capacity); + this(new HashSet(), new HashSet(), new HashSet(), new HashSet(), new HashSet(), capacity); } - private ClientState(Set activeTasks, Set standbyTasks, Set assignedTasks, Set prevActiveTasks, Set prevAssignedTasks, int capacity) { + private ClientState(Set activeTasks, Set standbyTasks, Set assignedTasks, Set prevActiveTasks, Set prevAssignedTasks, int capacity) { this.activeTasks = activeTasks; this.standbyTasks = standbyTasks; this.assignedTasks = assignedTasks; @@ -46,12 +48,12 @@ private ClientState(Set activeTasks, Set standbyTasks, Set assignedTask this.capacity = capacity; } - public ClientState copy() { - return new ClientState<>(new HashSet<>(activeTasks), new HashSet<>(standbyTasks), new HashSet<>(assignedTasks), + public ClientState copy() { + return new ClientState(new HashSet<>(activeTasks), new HashSet<>(standbyTasks), new HashSet<>(assignedTasks), new HashSet<>(prevActiveTasks), new HashSet<>(prevAssignedTasks), capacity); } - public void assign(final T taskId, final boolean active) { + public void assign(final TaskId taskId, final boolean active) { if (active) { activeTasks.add(taskId); } else { @@ -61,11 +63,11 @@ public void assign(final T taskId, final boolean active) { assignedTasks.add(taskId); } - public Set activeTasks() { + public Set activeTasks() { return activeTasks; } - public Set standbyTasks() { + public Set standbyTasks() { return standbyTasks; } @@ -81,31 +83,31 @@ public int activeTaskCount() { return activeTasks.size(); } - public void addPreviousActiveTasks(final Set prevTasks) { + public void addPreviousActiveTasks(final Set prevTasks) { prevActiveTasks.addAll(prevTasks); prevAssignedTasks.addAll(prevTasks); } - public void addPreviousStandbyTasks(final Set standbyTasks) { + public void addPreviousStandbyTasks(final Set standbyTasks) { prevAssignedTasks.addAll(standbyTasks); } @Override public String toString() { return "[activeTasks: (" + activeTasks + - ") standbyTasks: (" + standbyTasks + - ") assignedTasks: (" + assignedTasks + - ") prevActiveTasks: (" + prevActiveTasks + - ") prevAssignedTasks: (" + prevAssignedTasks + - ") capacity: " + capacity + - "]"; + ") standbyTasks: (" + standbyTasks + + ") assignedTasks: (" + assignedTasks + + ") prevActiveTasks: (" + prevActiveTasks + + ") prevAssignedTasks: (" + prevAssignedTasks + + ") capacity: " + capacity + + "]"; } boolean reachedCapacity() { return assignedTasks.size() >= capacity; } - boolean hasMoreAvailableCapacityThan(final ClientState other) { + boolean hasMoreAvailableCapacityThan(final ClientState other) { if (this.capacity <= 0) { throw new IllegalStateException("Capacity of this ClientState must be greater than 0."); } @@ -125,26 +127,26 @@ else if (thisLoad > otherLoad) return capacity > other.capacity; } - Set previousStandbyTasks() { - final Set standby = new HashSet<>(prevAssignedTasks); + Set previousStandbyTasks() { + final Set standby = new HashSet<>(prevAssignedTasks); standby.removeAll(prevActiveTasks); return standby; } - Set previousActiveTasks() { + Set previousActiveTasks() { return prevActiveTasks; } - boolean hasAssignedTask(final T taskId) { + boolean hasAssignedTask(final TaskId taskId) { return assignedTasks.contains(taskId); } // Visible for testing - Set assignedTasks() { + Set assignedTasks() { return assignedTasks; } - Set previousAssignedTasks() { + Set previousAssignedTasks() { return prevAssignedTasks; } @@ -155,4 +157,4 @@ int capacity() { boolean hasUnfulfilledQuota(final int tasksPerThread) { return activeTasks.size() < capacity * tasksPerThread; } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java index 81c9305d369fb..91738e0da20bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java @@ -31,13 +31,13 @@ public class StickyTaskAssignor implements TaskAssignor { private static final Logger log = LoggerFactory.getLogger(StickyTaskAssignor.class); - private final Map> clients; + private final Map clients; private final Set taskIds; private final Map previousActiveTaskAssignment = new HashMap<>(); private final Map> previousStandbyTaskAssignment = new HashMap<>(); private final TaskPairs taskPairs; - public StickyTaskAssignor(final Map> clients, final Set taskIds) { + public StickyTaskAssignor(final Map clients, final Set taskIds) { this.clients = clients; this.taskIds = taskIds; taskPairs = new TaskPairs(taskIds.size() * (taskIds.size() - 1) / 2); @@ -78,7 +78,7 @@ private void assignActive() { for (final Map.Entry entry : previousActiveTaskAssignment.entrySet()) { final TaskId taskId = entry.getKey(); if (taskIds.contains(taskId)) { - final ClientState client = clients.get(entry.getValue()); + final ClientState client = clients.get(entry.getValue()); if (client.hasUnfulfilledQuota(tasksPerThread)) { assignTaskToClient(assigned, taskId, client); } @@ -95,7 +95,7 @@ private void assignActive() { final Set clientIds = previousStandbyTaskAssignment.get(taskId); if (clientIds != null) { for (final ID clientId : clientIds) { - final ClientState client = clients.get(clientId); + final ClientState client = clients.get(clientId); if (client.hasUnfulfilledQuota(tasksPerThread)) { assignTaskToClient(assigned, taskId, client); iterator.remove(); @@ -115,12 +115,12 @@ private void assignActive() { private void allocateTaskWithClientCandidates(final TaskId taskId, final Set clientsWithin, final boolean active) { - final ClientState client = findClient(taskId, clientsWithin); + final ClientState client = findClient(taskId, clientsWithin); taskPairs.addPairs(taskId, client.assignedTasks()); client.assign(taskId, active); } - private void assignTaskToClient(final Set assigned, final TaskId taskId, final ClientState client) { + private void assignTaskToClient(final Set assigned, final TaskId taskId, final ClientState client) { taskPairs.addPairs(taskId, client.assignedTasks()); client.assign(taskId, true); assigned.add(taskId); @@ -128,7 +128,7 @@ private void assignTaskToClient(final Set assigned, final TaskId taskId, private Set findClientsWithoutAssignedTask(final TaskId taskId) { final Set clientIds = new HashSet<>(); - for (final Map.Entry> client : clients.entrySet()) { + for (final Map.Entry client : clients.entrySet()) { if (!client.getValue().hasAssignedTask(taskId)) { clientIds.add(client.getKey()); } @@ -137,20 +137,20 @@ private Set findClientsWithoutAssignedTask(final TaskId taskId) { } - private ClientState findClient(final TaskId taskId, - final Set clientsWithin) { + private ClientState findClient(final TaskId taskId, final Set clientsWithin) { + // optimize the case where there is only 1 id to search within. if (clientsWithin.size() == 1) { return clients.get(clientsWithin.iterator().next()); } - final ClientState previous = findClientsWithPreviousAssignedTask(taskId, clientsWithin); + final ClientState previous = findClientsWithPreviousAssignedTask(taskId, clientsWithin); if (previous == null) { return leastLoaded(taskId, clientsWithin); } if (shouldBalanceLoad(previous)) { - final ClientState standby = findLeastLoadedClientWithPreviousStandByTask(taskId, clientsWithin); + final ClientState standby = findLeastLoadedClientWithPreviousStandByTask(taskId, clientsWithin); if (standby == null || shouldBalanceLoad(standby)) { return leastLoaded(taskId, clientsWithin); @@ -161,12 +161,12 @@ private ClientState findClient(final TaskId taskId, return previous; } - private boolean shouldBalanceLoad(final ClientState client) { + private boolean shouldBalanceLoad(final ClientState client) { return client.reachedCapacity() && hasClientsWithMoreAvailableCapacity(client); } - private boolean hasClientsWithMoreAvailableCapacity(final ClientState client) { - for (ClientState clientState : clients.values()) { + private boolean hasClientsWithMoreAvailableCapacity(final ClientState client) { + for (ClientState clientState : clients.values()) { if (clientState.hasMoreAvailableCapacityThan(client)) { return true; } @@ -174,7 +174,7 @@ private boolean hasClientsWithMoreAvailableCapacity(final ClientState cl return false; } - private ClientState findClientsWithPreviousAssignedTask(final TaskId taskId, + private ClientState findClientsWithPreviousAssignedTask(final TaskId taskId, final Set clientsWithin) { final ID previous = previousActiveTaskAssignment.get(taskId); if (previous != null && clientsWithin.contains(previous)) { @@ -183,7 +183,7 @@ private ClientState findClientsWithPreviousAssignedTask(final TaskId tas return findLeastLoadedClientWithPreviousStandByTask(taskId, clientsWithin); } - private ClientState findLeastLoadedClientWithPreviousStandByTask(final TaskId taskId, final Set clientsWithin) { + private ClientState findLeastLoadedClientWithPreviousStandByTask(final TaskId taskId, final Set clientsWithin) { final Set ids = previousStandbyTaskAssignment.get(taskId); if (ids == null) { return null; @@ -193,20 +193,20 @@ private ClientState findLeastLoadedClientWithPreviousStandByTask(final T return leastLoaded(taskId, constrainTo); } - private ClientState leastLoaded(final TaskId taskId, final Set clientIds) { - final ClientState leastLoaded = findLeastLoaded(taskId, clientIds, true); + private ClientState leastLoaded(final TaskId taskId, final Set clientIds) { + final ClientState leastLoaded = findLeastLoaded(taskId, clientIds, true); if (leastLoaded == null) { return findLeastLoaded(taskId, clientIds, false); } return leastLoaded; } - private ClientState findLeastLoaded(final TaskId taskId, + private ClientState findLeastLoaded(final TaskId taskId, final Set clientIds, boolean checkTaskPairs) { - ClientState leastLoaded = null; + ClientState leastLoaded = null; for (final ID id : clientIds) { - final ClientState client = clients.get(id); + final ClientState client = clients.get(id); if (client.assignedTaskCount() == 0) { return client; } @@ -224,8 +224,8 @@ private ClientState findLeastLoaded(final TaskId taskId, } - private void mapPreviousTaskAssignment(final Map> clients) { - for (final Map.Entry> clientState : clients.entrySet()) { + private void mapPreviousTaskAssignment(final Map clients) { + for (final Map.Entry clientState : clients.entrySet()) { for (final TaskId activeTask : clientState.getValue().previousActiveTasks()) { previousActiveTaskAssignment.put(activeTask, clientState.getKey()); } @@ -240,9 +240,9 @@ private void mapPreviousTaskAssignment(final Map> client } - private int sumCapacity(final Collection> values) { + private int sumCapacity(final Collection values) { int capacity = 0; - for (ClientState client : values) { + for (ClientState client : values) { capacity += client.capacity(); } return capacity; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java index af2c9e34d64be..d0743f18cb28c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.processor.TaskId; import org.junit.Test; import java.util.Collections; @@ -28,7 +29,7 @@ public class ClientStateTest { - private final ClientState client = new ClientState<>(1); + private final ClientState client = new ClientState(1); @Test public void shouldHaveNotReachedCapacityWhenAssignedTasksLessThanCapacity() throws Exception { @@ -37,86 +38,101 @@ public void shouldHaveNotReachedCapacityWhenAssignedTasksLessThanCapacity() thro @Test public void shouldHaveReachedCapacityWhenAssignedTasksGreaterThanOrEqualToCapacity() throws Exception { - client.assign(1, true); + client.assign(new TaskId(0, 1), true); assertTrue(client.reachedCapacity()); } @Test public void shouldAddActiveTasksToBothAssignedAndActive() throws Exception { - client.assign(1, true); - assertThat(client.activeTasks(), equalTo(Collections.singleton(1))); - assertThat(client.assignedTasks(), equalTo(Collections.singleton(1))); + final TaskId tid = new TaskId(0, 1); + + client.assign(tid, true); + assertThat(client.activeTasks(), equalTo(Collections.singleton(tid))); + assertThat(client.assignedTasks(), equalTo(Collections.singleton(tid))); assertThat(client.assignedTaskCount(), equalTo(1)); assertThat(client.standbyTasks().size(), equalTo(0)); } @Test public void shouldAddStandbyTasksToBothStandbyAndActive() throws Exception { - client.assign(1, false); - assertThat(client.assignedTasks(), equalTo(Collections.singleton(1))); - assertThat(client.standbyTasks(), equalTo(Collections.singleton(1))); + final TaskId tid = new TaskId(0, 1); + + client.assign(tid, false); + assertThat(client.assignedTasks(), equalTo(Collections.singleton(tid))); + assertThat(client.standbyTasks(), equalTo(Collections.singleton(tid))); assertThat(client.assignedTaskCount(), equalTo(1)); assertThat(client.activeTasks().size(), equalTo(0)); } @Test public void shouldAddPreviousActiveTasksToPreviousAssignedAndPreviousActive() throws Exception { - client.addPreviousActiveTasks(Utils.mkSet(1, 2)); - assertThat(client.previousActiveTasks(), equalTo(Utils.mkSet(1, 2))); - assertThat(client.previousAssignedTasks(), equalTo(Utils.mkSet(1, 2))); + final TaskId tid1 = new TaskId(0, 1); + final TaskId tid2 = new TaskId(0, 2); + + client.addPreviousActiveTasks(Utils.mkSet(tid1, tid2)); + assertThat(client.previousActiveTasks(), equalTo(Utils.mkSet(tid1, tid2))); + assertThat(client.previousAssignedTasks(), equalTo(Utils.mkSet(tid1, tid2))); } @Test public void shouldAddPreviousStandbyTasksToPreviousAssigned() throws Exception { - client.addPreviousStandbyTasks(Utils.mkSet(1, 2)); + final TaskId tid1 = new TaskId(0, 1); + final TaskId tid2 = new TaskId(0, 2); + + client.addPreviousStandbyTasks(Utils.mkSet(tid1, tid2)); assertThat(client.previousActiveTasks().size(), equalTo(0)); - assertThat(client.previousAssignedTasks(), equalTo(Utils.mkSet(1, 2))); + assertThat(client.previousAssignedTasks(), equalTo(Utils.mkSet(tid1, tid2))); } @Test public void shouldHaveAssignedTaskIfActiveTaskAssigned() throws Exception { - client.assign(2, true); - assertTrue(client.hasAssignedTask(2)); + final TaskId tid = new TaskId(0, 2); + + client.assign(tid, true); + assertTrue(client.hasAssignedTask(tid)); } @Test public void shouldHaveAssignedTaskIfStandbyTaskAssigned() throws Exception { - client.assign(2, false); - assertTrue(client.hasAssignedTask(2)); + final TaskId tid = new TaskId(0, 2); + + client.assign(tid, false); + assertTrue(client.hasAssignedTask(tid)); } @Test public void shouldNotHaveAssignedTaskIfTaskNotAssigned() throws Exception { - client.assign(2, true); - assertFalse(client.hasAssignedTask(3)); + + client.assign(new TaskId(0, 2), true); + assertFalse(client.hasAssignedTask(new TaskId(0, 3))); } @Test public void shouldHaveMoreAvailableCapacityWhenCapacityTheSameButFewerAssignedTasks() throws Exception { - final ClientState c2 = new ClientState<>(1); - client.assign(1, true); + final ClientState c2 = new ClientState(1); + client.assign(new TaskId(0, 1), true); assertTrue(c2.hasMoreAvailableCapacityThan(client)); assertFalse(client.hasMoreAvailableCapacityThan(c2)); } @Test public void shouldHaveMoreAvailableCapacityWhenCapacityHigherAndSameAssignedTaskCount() throws Exception { - final ClientState c2 = new ClientState<>(2); + final ClientState c2 = new ClientState(2); assertTrue(c2.hasMoreAvailableCapacityThan(client)); assertFalse(client.hasMoreAvailableCapacityThan(c2)); } @Test public void shouldUseMultiplesOfCapacityToDetermineClientWithMoreAvailableCapacity() throws Exception { - final ClientState c2 = new ClientState<>(2); + final ClientState c2 = new ClientState(2); for (int i = 0; i < 7; i++) { - c2.assign(i, true); + c2.assign(new TaskId(0, i), true); } for (int i = 7; i < 11; i++) { - client.assign(i, true); + client.assign(new TaskId(0, i), true); } assertTrue(c2.hasMoreAvailableCapacityThan(client)); @@ -124,39 +140,39 @@ public void shouldUseMultiplesOfCapacityToDetermineClientWithMoreAvailableCapaci @Test public void shouldHaveMoreAvailableCapacityWhenCapacityIsTheSameButAssignedTasksIsLess() throws Exception { - final ClientState c1 = new ClientState<>(3); - final ClientState c2 = new ClientState<>(3); + final ClientState c1 = new ClientState(3); + final ClientState c2 = new ClientState(3); for (int i = 0; i < 4; i++) { - c1.assign(i, true); - c2.assign(i, true); + c1.assign(new TaskId(0, i), true); + c2.assign(new TaskId(0, i), true); } - c2.assign(5, true); + c2.assign(new TaskId(0, 5), true); assertTrue(c1.hasMoreAvailableCapacityThan(c2)); } @Test(expected = IllegalStateException.class) public void shouldThrowIllegalStateExceptionIfCapacityOfThisClientStateIsZero() throws Exception { - final ClientState c1 = new ClientState<>(0); - c1.hasMoreAvailableCapacityThan(new ClientState(1)); + final ClientState c1 = new ClientState(0); + c1.hasMoreAvailableCapacityThan(new ClientState(1)); } @Test(expected = IllegalStateException.class) public void shouldThrowIllegalStateExceptionIfCapacityOfOtherClientStateIsZero() throws Exception { - final ClientState c1 = new ClientState<>(1); - c1.hasMoreAvailableCapacityThan(new ClientState(0)); + final ClientState c1 = new ClientState(1); + c1.hasMoreAvailableCapacityThan(new ClientState(0)); } @Test public void shouldHaveUnfulfilledQuotaWhenActiveTaskSizeLessThanCapacityTimesTasksPerThread() throws Exception { - final ClientState client = new ClientState<>(1); - client.assign(1, true); + final ClientState client = new ClientState(1); + client.assign(new TaskId(0, 1), true); assertTrue(client.hasUnfulfilledQuota(2)); } @Test public void shouldNotHaveUnfulfilledQuotaWhenActiveTaskSizeGreaterEqualThanCapacityTimesTasksPerThread() throws Exception { - final ClientState client = new ClientState<>(1); - client.assign(1, true); + final ClientState client = new ClientState(1); + client.assign(new TaskId(0, 1), true); assertFalse(client.hasUnfulfilledQuota(1)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java index f37bf7d6a9a0c..449dabd157efe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java @@ -44,7 +44,7 @@ public class StickyTaskAssignorTest { private final TaskId task03 = new TaskId(0, 3); private final TaskId task04 = new TaskId(0, 4); private final TaskId task05 = new TaskId(0, 5); - private final Map> clients = new TreeMap<>(); + private final Map clients = new TreeMap<>(); private final Integer p1 = 1; private final Integer p2 = 2; private final Integer p3 = 3; @@ -153,11 +153,11 @@ public void shouldKeepActiveTaskStickynessWhenMoreClientThanActiveTasks() { @Test public void shouldAssignTasksToClientWithPreviousStandbyTasks() throws Exception { - final ClientState client1 = createClient(p1, 1); + final ClientState client1 = createClient(p1, 1); client1.addPreviousStandbyTasks(Utils.mkSet(task02)); - final ClientState client2 = createClient(p2, 1); + final ClientState client2 = createClient(p2, 1); client2.addPreviousStandbyTasks(Utils.mkSet(task01)); - final ClientState client3 = createClient(p3, 1); + final ClientState client3 = createClient(p3, 1); client3.addPreviousStandbyTasks(Utils.mkSet(task00)); final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task01, task02); @@ -171,9 +171,9 @@ public void shouldAssignTasksToClientWithPreviousStandbyTasks() throws Exception @Test public void shouldAssignBasedOnCapacityWhenMultipleClientHaveStandbyTasks() throws Exception { - final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00); + final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00); c1.addPreviousStandbyTasks(Utils.mkSet(task01)); - final ClientState c2 = createClientWithPreviousActiveTasks(p2, 2, task02); + final ClientState c2 = createClientWithPreviousActiveTasks(p2, 2, task02); c2.addPreviousStandbyTasks(Utils.mkSet(task01)); final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task01, task02); @@ -290,7 +290,7 @@ public void shouldBalanceActiveAndStandbyTasksAcrossAvailableClients() throws Ex final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task01, task02); taskAssignor.assign(1); - for (final ClientState clientState : clients.values()) { + for (final ClientState clientState : clients.values()) { assertThat(clientState.assignedTaskCount(), equalTo(1)); } } @@ -365,9 +365,9 @@ public void shouldNotHaveSameAssignmentOnAnyTwoHostsWhenThereArePreviousActiveTa @Test public void shouldNotHaveSameAssignmentOnAnyTwoHostsWhenThereArePreviousStandbyTasks() throws Exception { - final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task02); + final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task02); c1.addPreviousStandbyTasks(Utils.mkSet(task03, task00)); - final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task03, task00); + final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task03, task00); c2.addPreviousStandbyTasks(Utils.mkSet(task01, task02)); createClient(p3, 1); @@ -487,14 +487,14 @@ public void shouldAssignTasksNotPreviouslyActiveToNewClient() throws Exception { final TaskId task22 = new TaskId(2, 2); final TaskId task23 = new TaskId(2, 3); - final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task12, task13); + final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task12, task13); c1.addPreviousStandbyTasks(Utils.mkSet(task00, task11, task20, task21, task23)); - final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task00, task11, task22); + final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task00, task11, task22); c2.addPreviousStandbyTasks(Utils.mkSet(task01, task10, task02, task20, task03, task12, task21, task13, task23)); - final ClientState c3 = createClientWithPreviousActiveTasks(p3, 1, task20, task21, task23); + final ClientState c3 = createClientWithPreviousActiveTasks(p3, 1, task20, task21, task23); c3.addPreviousStandbyTasks(Utils.mkSet(task02, task12)); - final ClientState newClient = createClient(p4, 1); + final ClientState newClient = createClient(p4, 1); newClient.addPreviousStandbyTasks(Utils.mkSet(task00, task10, task01, task02, task11, task20, task03, task12, task21, task13, task22, task23)); final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task10, task01, task02, task11, task20, task03, task12, task21, task13, task22, task23); @@ -517,15 +517,15 @@ public void shouldAssignTasksNotPreviouslyActiveToMultipleNewClients() throws Ex final TaskId task22 = new TaskId(2, 2); final TaskId task23 = new TaskId(2, 3); - final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task12, task13); + final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task01, task12, task13); c1.addPreviousStandbyTasks(Utils.mkSet(task00, task11, task20, task21, task23)); - final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task00, task11, task22); + final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task00, task11, task22); c2.addPreviousStandbyTasks(Utils.mkSet(task01, task10, task02, task20, task03, task12, task21, task13, task23)); - final ClientState bounce1 = createClient(p3, 1); + final ClientState bounce1 = createClient(p3, 1); bounce1.addPreviousStandbyTasks(Utils.mkSet(task20, task21, task23)); - final ClientState bounce2 = createClient(p4, 1); + final ClientState bounce2 = createClient(p4, 1); bounce2.addPreviousStandbyTasks(Utils.mkSet(task02, task03, task10)); final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task10, task01, task02, task11, task20, task03, task12, task21, task13, task22, task23); @@ -547,9 +547,9 @@ public void shouldAssignTasksToNewClient() throws Exception { @Test public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExistingClients() throws Exception { - final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00, task01, task02); - final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task03, task04, task05); - final ClientState newClient = createClient(p3, 1); + final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00, task01, task02); + final ClientState c2 = createClientWithPreviousActiveTasks(p2, 1, task03, task04, task05); + final ClientState newClient = createClient(p3, 1); final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task01, task02, task03, task04, task05); taskAssignor.assign(0); @@ -567,10 +567,10 @@ public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExisting @Test public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExistingAndBouncedClients() throws Exception { final TaskId task06 = new TaskId(0, 6); - final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00, task01, task02, task06); - final ClientState c2 = createClient(p2, 1); + final ClientState c1 = createClientWithPreviousActiveTasks(p1, 1, task00, task01, task02, task06); + final ClientState c2 = createClient(p2, 1); c2.addPreviousStandbyTasks(Utils.mkSet(task03, task04, task05)); - final ClientState newClient = createClient(p3, 1); + final ClientState newClient = createClient(p3, 1); final StickyTaskAssignor taskAssignor = createTaskAssignor(task00, task01, task02, task03, task04, task05, task06); taskAssignor.assign(0); @@ -594,7 +594,7 @@ private StickyTaskAssignor createTaskAssignor(final TaskId... tasks) { private List allActiveTasks() { final List allActive = new ArrayList<>(); - for (final ClientState client : clients.values()) { + for (final ClientState client : clients.values()) { allActive.addAll(client.activeTasks()); } Collections.sort(allActive); @@ -603,19 +603,19 @@ private List allActiveTasks() { private List allStandbyTasks() { final List tasks = new ArrayList<>(); - for (final ClientState client : clients.values()) { + for (final ClientState client : clients.values()) { tasks.addAll(client.standbyTasks()); } Collections.sort(tasks); return tasks; } - private ClientState createClient(final Integer processId, final int capacity) { + private ClientState createClient(final Integer processId, final int capacity) { return createClientWithPreviousActiveTasks(processId, capacity); } - private ClientState createClientWithPreviousActiveTasks(final Integer processId, final int capacity, final TaskId... taskIds) { - final ClientState clientState = new ClientState<>(capacity); + private ClientState createClientWithPreviousActiveTasks(final Integer processId, final int capacity, final TaskId... taskIds) { + final ClientState clientState = new ClientState(capacity); clientState.addPreviousActiveTasks(Utils.mkSet(taskIds)); clients.put(processId, clientState); return clientState; From 0011a3dc337a41ceb21b1567db7bdc3219a07552 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Fri, 10 Mar 2017 13:43:04 -0800 Subject: [PATCH 088/101] MINOR: increase RocksDb parallelism Author: Eno Thereska Author: Eno Thereska Reviewers: Damian Guy, Guozhang Wang Closes #2663 from enothereska/minor-rocksdb-parallel --- .../apache/kafka/streams/state/internals/RocksDBStore.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 823ad47ee5463..932ddd29fa0b8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -124,6 +124,10 @@ public void openDB(ProcessorContext context) { options.setCreateIfMissing(true); options.setErrorIfExists(false); options.setInfoLogLevel(InfoLogLevel.ERROR_LEVEL); + // this is the recommended way to increase parallelism in RocksDb + // note that the current implementation increases the number of compaction threads + // but not flush threads. + options.setIncreaseParallelism(Runtime.getRuntime().availableProcessors()); wOptions = new WriteOptions(); wOptions.setDisableWAL(true); From 046519d0714591785cd984ab2d57e193b54a2f19 Mon Sep 17 00:00:00 2001 From: Aaron Coburn Date: Fri, 10 Mar 2017 16:48:48 -0800 Subject: [PATCH 089/101] MINOR: incorrect javadoc formatting When wrapped in a `{code ...}` block, `<>` are not formatted as `<>`. For instance, see the formatting of the example in https://kafka.apache.org/0102/javadoc/org/apache/kafka/streams/KafkaStreams.html Author: Aaron Coburn Reviewers: Matthias J. Sax, Guozhang Wang Closes #2671 from acoburn/javadoc_formatting --- .../src/main/java/org/apache/kafka/streams/KafkaStreams.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 2810ff884bc4e..8d8626da088c5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -96,7 +96,7 @@ *

    * A simple example might look like this: *

    {@code
    - * Map<String, Object> props = new HashMap<>();
    + * Map props = new HashMap<>();
      * props.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-stream-processing-application");
      * props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
      * props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
    @@ -104,7 +104,7 @@
      * StreamsConfig config = new StreamsConfig(props);
      *
      * KStreamBuilder builder = new KStreamBuilder();
    - * builder.stream("my-input-topic").mapValues(value -> value.length().toString()).to("my-output-topic");
    + * builder.stream("my-input-topic").mapValues(value -> value.length().toString()).to("my-output-topic");
      *
      * KafkaStreams streams = new KafkaStreams(builder, config);
      * streams.start();
    
    From cd69daa4150631e9cea4e299c5ea08e12359118d Mon Sep 17 00:00:00 2001
    From: "Michael G. Noll" 
    Date: Mon, 13 Mar 2017 11:49:49 -0700
    Subject: [PATCH 090/101] KAFKA-4769: Add Float serializer, deserializer, serde
    
    Author: Michael G. Noll 
    
    Reviewers: Dongjin Lee, Eno Thereska, Damian Guy, Colin P. McCabe, Matthas J. Sax, Guozhang Wang
    
    Closes #2554 from miguno/KAFKA-4769
    ---
     .../serialization/FloatDeserializer.java      | 47 ++++++++++
     .../common/serialization/FloatSerializer.java | 42 +++++++++
     .../kafka/common/serialization/Serdes.java    | 17 ++++
     .../serialization/SerializationTest.java      | 93 +++++++++++++++++++
     4 files changed, 199 insertions(+)
     create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java
     create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java
    
    diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java
    new file mode 100644
    index 0000000000000..d5f49f53aa8d0
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java
    @@ -0,0 +1,47 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
    + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
    + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
    + * License. You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
    + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
    + * specific language governing permissions and limitations under the License.
    + */
    +package org.apache.kafka.common.serialization;
    +
    +import org.apache.kafka.common.errors.SerializationException;
    +
    +import java.util.Map;
    +
    +public class FloatDeserializer implements Deserializer {
    +
    +    @Override
    +    public void configure(final Map configs, final boolean isKey) {
    +        // nothing to do
    +    }
    +
    +    @Override
    +    public Float deserialize(final String topic, final byte[] data) {
    +        if (data == null)
    +            return null;
    +        if (data.length != 4) {
    +            throw new SerializationException("Size of data received by Deserializer is not 4");
    +        }
    +
    +        int value = 0;
    +        for (byte b : data) {
    +            value <<= 8;
    +            value |= b & 0xFF;
    +        }
    +        return Float.intBitsToFloat(value);
    +    }
    +
    +    @Override
    +    public void close() {
    +        // nothing to do
    +    }
    +
    +}
    \ No newline at end of file
    diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java
    new file mode 100644
    index 0000000000000..badc258089828
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java
    @@ -0,0 +1,42 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
    + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
    + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
    + * License. You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
    + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
    + * specific language governing permissions and limitations under the License.
    + */
    +package org.apache.kafka.common.serialization;
    +
    +import java.util.Map;
    +
    +public class FloatSerializer implements Serializer {
    +
    +    @Override
    +    public void configure(final Map configs, final boolean isKey) {
    +        // nothing to do
    +    }
    +
    +    @Override
    +    public byte[] serialize(final String topic, final Float data) {
    +        if (data == null)
    +            return null;
    +
    +        long bits = Float.floatToRawIntBits(data);
    +        return new byte[] {
    +            (byte) (bits >>> 24),
    +            (byte) (bits >>> 16),
    +            (byte) (bits >>> 8),
    +            (byte) bits
    +        };
    +    }
    +
    +    @Override
    +    public void close() {
    +        // nothing to do
    +    }
    +}
    \ No newline at end of file
    diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java
    index 15f9748e246e1..0793321a017c0 100644
    --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java
    +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java
    @@ -70,6 +70,12 @@ public IntegerSerde() {
             }
         }
     
    +    static public final class FloatSerde extends WrapperSerde {
    +        public FloatSerde() {
    +            super(new FloatSerializer(), new FloatDeserializer());
    +        }
    +    }
    +
         static public final class DoubleSerde extends WrapperSerde {
             public DoubleSerde() {
                 super(new DoubleSerializer(), new DoubleDeserializer());
    @@ -114,6 +120,10 @@ static public  Serde serdeFrom(Class type) {
                 return (Serde) Long();
             }
     
    +        if (Float.class.isAssignableFrom(type)) {
    +            return (Serde) Float();
    +        }
    +
             if (Double.class.isAssignableFrom(type)) {
                 return (Serde) Double();
             }
    @@ -165,6 +175,13 @@ static public Serde Integer() {
             return new IntegerSerde();
         }
     
    +    /*
    +     * A serde for nullable {@code Float} type.
    +     */
    +    static public Serde Float() {
    +        return new FloatSerde();
    +    }
    +
         /*
          * A serde for nullable {@code Double} type.
          */
    diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java
    index 50b4594241409..12ccbe4acdf22 100644
    --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java
    +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java
    @@ -16,6 +16,7 @@
      */
     package org.apache.kafka.common.serialization;
     
    +import org.apache.kafka.common.errors.SerializationException;
     import org.junit.Test;
     
     import java.nio.ByteBuffer;
    @@ -24,7 +25,11 @@
     import java.util.List;
     import java.util.Map;
     
    +import static org.hamcrest.CoreMatchers.equalTo;
    +import static org.hamcrest.MatcherAssert.assertThat;
    +import static org.hamcrest.core.IsNull.nullValue;
     import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.fail;
     
     public class SerializationTest {
     
    @@ -122,6 +127,94 @@ public void testLongSerializer() {
             deserializer.close();
         }
     
    +    @Test
    +    public void shouldSerializeDeserializeFloat() {
    +        final Float[] floats = new Float[]{
    +            5678567.12312f,
    +            -5678567.12341f
    +        };
    +        final Serializer serializer = Serdes.Float().serializer();
    +        final Deserializer deserializer = Serdes.Float().deserializer();
    +
    +        for (final Float value : floats) {
    +            assertThat("Should round-trip a float",
    +                value, equalTo(deserializer.deserialize(topic, serializer.serialize(topic, value))));
    +        }
    +
    +        serializer.close();
    +        deserializer.close();
    +    }
    +
    +    @Test
    +    public void floatSerializerShouldReturnNullForNull() {
    +        final Serializer serializer = Serdes.Float().serializer();
    +        assertThat(serializer.serialize(topic, null), nullValue());
    +        serializer.close();
    +    }
    +
    +    @Test
    +    public void floatDeserializerShouldReturnNullForNull() {
    +        final Deserializer deserializer = Serdes.Float().deserializer();
    +        assertThat(deserializer.deserialize(topic, null), nullValue());
    +        deserializer.close();
    +    }
    +
    +    @Test
    +    public void floatDeserializerShouldThrowSerializationExceptionOnZeroBytes() {
    +        final Deserializer deserializer = Serdes.Float().deserializer();
    +        try {
    +            deserializer.deserialize(topic, new byte[0]);
    +            fail("Should have thrown a SerializationException because of zero input bytes");
    +        } catch (SerializationException e) {
    +            // Ignore (there's no contract on the details of the exception)
    +        }
    +        deserializer.close();
    +    }
    +
    +    @Test
    +    public void floatDeserializerShouldThrowSerializationExceptionOnTooFewBytes() {
    +        final Deserializer deserializer = Serdes.Float().deserializer();
    +        try {
    +            deserializer.deserialize(topic, new byte[3]);
    +            fail("Should have thrown a SerializationException because of too few input bytes");
    +        } catch (SerializationException e) {
    +            // Ignore (there's no contract on the details of the exception)
    +        }
    +        deserializer.close();
    +    }
    +
    +
    +    @Test
    +    public void floatDeserializerShouldThrowSerializationExceptionOnTooManyBytes() {
    +        final Deserializer deserializer = Serdes.Float().deserializer();
    +        try {
    +            deserializer.deserialize(topic, new byte[5]);
    +            fail("Should have thrown a SerializationException because of too many input bytes");
    +        } catch (SerializationException e) {
    +            // Ignore (there's no contract on the details of the exception)
    +        }
    +        deserializer.close();
    +    }
    +
    +    @Test
    +    public void floatSerdeShouldPreserveNaNValues() {
    +        final int someNaNAsIntBits = 0x7f800001;
    +        final float someNaN = Float.intBitsToFloat(someNaNAsIntBits);
    +        final int anotherNaNAsIntBits = 0x7f800002;
    +        final float anotherNaN = Float.intBitsToFloat(anotherNaNAsIntBits);
    +
    +        final Serde serde = Serdes.Float();
    +        // Because of NaN semantics we must assert based on the raw int bits.
    +        final Float roundtrip = serde.deserializer().deserialize(topic,
    +            serde.serializer().serialize(topic, someNaN));
    +        assertThat(Float.floatToRawIntBits(roundtrip), equalTo(someNaNAsIntBits));
    +        final Float otherRoundtrip = serde.deserializer().deserialize(topic,
    +            serde.serializer().serialize(topic, anotherNaN));
    +        assertThat(Float.floatToRawIntBits(otherRoundtrip), equalTo(anotherNaNAsIntBits));
    +
    +        serde.close();
    +    }
    +
         @Test
         public void testDoubleSerializer() {
             Double[] doubles = new Double[]{
    
    From 85e7a0a2b0d56891c13c15fc88a3011877f9c9e9 Mon Sep 17 00:00:00 2001
    From: Andrey Dyachkov 
    Date: Mon, 13 Mar 2017 12:05:47 -0700
    Subject: [PATCH 091/101] KAFKA-4657: Improve test coverage of
     CompositeReadOnlyWindowStore
    
    This commmit brings improved test coverage for window store fetch method
    and WindowStoreIterator
    
    Author: Andrey Dyachkov 
    
    Reviewers: Damian Guy, Guozhang Wang
    
    Closes #2672 from adyach/trunk
    ---
     .../CompositeReadOnlyWindowStoreTest.java     | 49 +++++++++++++++++--
     1 file changed, 46 insertions(+), 3 deletions(-)
    
    diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
    index d0c82e46f8aa7..6f4ff07f8212f 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java
    @@ -22,12 +22,16 @@
     import org.apache.kafka.streams.state.WindowStoreIterator;
     import org.apache.kafka.test.StateStoreProviderStub;
     import org.apache.kafka.test.StreamsTestUtils;
    +import org.junit.Assert;
     import org.junit.Before;
    +import org.junit.Rule;
     import org.junit.Test;
    +import org.junit.rules.ExpectedException;
     
     import java.util.Arrays;
     import java.util.Collections;
     import java.util.List;
    +import java.util.NoSuchElementException;
     
     import static java.util.Arrays.asList;
     import static org.junit.Assert.assertEquals;
    @@ -43,6 +47,9 @@ public class CompositeReadOnlyWindowStoreTest {
         private ReadOnlyWindowStoreStub
                 otherUnderlyingStore;
     
    +    @Rule
    +    public final ExpectedException windowStoreIteratorException = ExpectedException.none();
    +
         @Before
         public void before() {
             stubProviderOne = new StateStoreProviderStub(false);
    @@ -106,17 +113,53 @@ public void shouldNotGetValuesFromOtherStores() throws Exception {
             assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results);
         }
     
    -
         @Test(expected = InvalidStateStoreException.class)
         public void shouldThrowInvalidStateStoreExceptionOnRebalance() throws Exception {
             final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new StateStoreProviderStub(true), QueryableStoreTypes.windowStore(), "foo");
             store.fetch("key", 1, 10);
         }
     
    -    @Test(expected = InvalidStateStoreException.class)
    +    @Test
         public void shouldThrowInvalidStateStoreExceptionIfFetchThrows() throws Exception {
             underlyingWindowStore.setOpen(false);
    -        underlyingWindowStore.fetch("key", 1, 10);
    +        final CompositeReadOnlyWindowStore store =
    +                new CompositeReadOnlyWindowStore<>(stubProviderOne, QueryableStoreTypes.windowStore(), "window-store");
    +        try {
    +            store.fetch("key", 1, 10);
    +            Assert.fail("InvalidStateStoreException was expected");
    +        } catch (InvalidStateStoreException e) {
    +            Assert.assertEquals("State store is not available anymore and may have been migrated to another instance; " +
    +                    "please re-discover its location from the state metadata.", e.getMessage());
    +        }
    +    }
    +
    +    @Test
    +    public void emptyIteratorAlwaysReturnsFalse() throws Exception {
    +        final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new
    +                StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo");
    +        final WindowStoreIterator windowStoreIterator = store.fetch("key", 1, 10);
    +
    +        Assert.assertFalse(windowStoreIterator.hasNext());
    +    }
    +
    +    @Test
    +    public void emptyIteratorPeekNextKeyShouldThrowNoSuchElementException() throws Exception {
    +        final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new
    +                StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo");
    +        final WindowStoreIterator windowStoreIterator = store.fetch("key", 1, 10);
    +
    +        windowStoreIteratorException.expect(NoSuchElementException.class);
    +        windowStoreIterator.peekNextKey();
    +    }
    +
    +    @Test
    +    public void emptyIteratorNextShouldThrowNoSuchElementException() throws Exception {
    +        final CompositeReadOnlyWindowStore store = new CompositeReadOnlyWindowStore<>(new
    +                StateStoreProviderStub(false), QueryableStoreTypes.windowStore(), "foo");
    +        final WindowStoreIterator windowStoreIterator = store.fetch("key", 1, 10);
    +
    +        windowStoreIteratorException.expect(NoSuchElementException.class);
    +        windowStoreIterator.next();
         }
     
     }
    \ No newline at end of file
    
    From 2a9c18a87d4374d03f8a07b37a4e66d74325f1c2 Mon Sep 17 00:00:00 2001
    From: Onur Karaman 
    Date: Mon, 13 Mar 2017 13:00:53 -0700
    Subject: [PATCH 092/101] KAFKA-4891; kafka.request.logger TRACE regression
    
    Both the headers and requests have regressed to just show object ids instead of their contents from their underlying structs. I'm guessing this regression came from commit [fc1cfe475e8ae8458d8ddf119ce18d0c64653a70](https://github.com/apache/kafka/commit/fc1cfe475e8ae8458d8ddf119ce18d0c64653a70)
    
    Author: Onur Karaman 
    
    Reviewers: Jiangjie Qin 
    
    Closes #2678 from onurkaraman/KAFKA-4891
    ---
     .../org/apache/kafka/common/requests/AbstractRequest.java    | 5 +++++
     .../java/org/apache/kafka/common/requests/RequestHeader.java | 5 +++++
     2 files changed, 10 insertions(+)
    
    diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
    index c2ad39db3deb7..3d79f7e48cbe8 100644
    --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
    +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
    @@ -79,6 +79,11 @@ public ByteBuffer serialize(RequestHeader header) {
     
         protected abstract Struct toStruct();
     
    +    @Override
    +    public String toString() {
    +        return toStruct().toString();
    +    }
    +
         /**
          * Get an error response for a request
          */
    diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
    index 4b818e71cbdbf..8a0a549b93c2d 100644
    --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
    +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
    @@ -85,4 +85,9 @@ public ResponseHeader toResponseHeader() {
         public static RequestHeader parse(ByteBuffer buffer) {
             return new RequestHeader(Protocol.REQUEST_HEADER.read(buffer));
         }
    +
    +    @Override
    +    public String toString() {
    +        return toStruct().toString();
    +    }
     }
    
    From d3b8ff02437e735801e36cc4970b794344400058 Mon Sep 17 00:00:00 2001
    From: Guozhang Wang 
    Date: Mon, 13 Mar 2017 13:47:17 -0700
    Subject: [PATCH 093/101] HOTFIX: Fix apache headers in float serde class files
    
    hachikuji
    
    Author: Guozhang Wang 
    
    Reviewers: Jason Gustafson 
    
    Closes #2679 from guozhangwang/KHotfix-serde-headers
    ---
     .../serialization/FloatDeserializer.java      | 22 +++++++++++--------
     .../common/serialization/FloatSerializer.java | 22 +++++++++++--------
     2 files changed, 26 insertions(+), 18 deletions(-)
    
    diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java
    index d5f49f53aa8d0..3834ce20b079b 100644
    --- a/clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java
    +++ b/clients/src/main/java/org/apache/kafka/common/serialization/FloatDeserializer.java
    @@ -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 org.apache.kafka.common.serialization;
     
    diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java
    index badc258089828..6eb766dcd4254 100644
    --- a/clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java
    +++ b/clients/src/main/java/org/apache/kafka/common/serialization/FloatSerializer.java
    @@ -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 org.apache.kafka.common.serialization;
     
    
    From 9e4548df30d50a56ae99cb3383f1a3f97bbe77bb Mon Sep 17 00:00:00 2001
    From: Damian Guy 
    Date: Tue, 14 Mar 2017 14:21:03 -0700
    Subject: [PATCH 094/101] KAFKA-4863; Querying window store may return unwanted
     keys
    
    Make sure that the iterator returned from `WindowStore.fetch(..)` only returns matching keys, rather than all keys that are a prefix match.
    
    Author: Damian Guy 
    
    Reviewers: Eno Thereska, Guozhang Wang
    
    Closes #2662 from dguy/kafka-4863
    ---
     .../state/internals/CachingSessionStore.java  |  52 --
     .../state/internals/CachingWindowStore.java   |  12 +-
     .../internals/FilteredCacheIterator.java      |  73 ++
     .../state/internals/WindowKeySchema.java      |  22 +-
     .../internals/CachingWindowStoreTest.java     |  25 +-
     .../internals/FilteredCacheIteratorTest.java  | 118 +++
     .../internals/RocksDBWindowStoreTest.java     | 843 +++++++++---------
     7 files changed, 663 insertions(+), 482 deletions(-)
     create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java
     create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
    index 58c013328741c..a4b46ff2ff97e 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java
    @@ -18,7 +18,6 @@
     
     import org.apache.kafka.common.serialization.Serde;
     import org.apache.kafka.common.utils.Bytes;
    -import org.apache.kafka.streams.KeyValue;
     import org.apache.kafka.streams.kstream.Windowed;
     import org.apache.kafka.streams.kstream.internals.CacheFlushListener;
     import org.apache.kafka.streams.kstream.internals.SessionKeySerde;
    @@ -31,7 +30,6 @@
     import org.apache.kafka.streams.state.StateSerdes;
     
     import java.util.List;
    -import java.util.NoSuchElementException;
     
     
     class CachingSessionStore extends WrappedStateStore.AbstractStateStore implements SessionStore, CachedStateStore, AGG> {
    @@ -162,54 +160,4 @@ public void setFlushListener(CacheFlushListener, AGG> flushListener)
             this.flushListener = flushListener;
         }
     
    -    private static class FilteredCacheIterator implements PeekingKeyValueIterator {
    -        private final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator;
    -        private final HasNextCondition hasNextCondition;
    -
    -        FilteredCacheIterator(final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator,
    -                              final HasNextCondition hasNextCondition) {
    -            this.cacheIterator = cacheIterator;
    -            this.hasNextCondition = hasNextCondition;
    -        }
    -
    -        @Override
    -        public void close() {
    -            // no-op
    -        }
    -
    -        @Override
    -        public Bytes peekNextKey() {
    -            if (!hasNext()) {
    -                throw new NoSuchElementException();
    -            }
    -            return cacheIterator.peekNextKey();
    -        }
    -
    -        @Override
    -        public boolean hasNext() {
    -            return hasNextCondition.hasNext(cacheIterator);
    -        }
    -
    -        @Override
    -        public KeyValue next() {
    -            if (!hasNext()) {
    -                throw new NoSuchElementException();
    -            }
    -            return cacheIterator.next();
    -
    -        }
    -
    -        @Override
    -        public void remove() {
    -            throw new UnsupportedOperationException();
    -        }
    -
    -        @Override
    -        public KeyValue peekNext() {
    -            if (!hasNext()) {
    -                throw new NoSuchElementException();
    -            }
    -            return cacheIterator.peekNext();
    -        }
    -    }
     }
    diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
    index 7ea2fa4af3498..4003e546dca7d 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java
    @@ -38,6 +38,7 @@ class CachingWindowStore extends WrappedStateStore.AbstractStateStore impl
         private final Serde keySerde;
         private final Serde valueSerde;
         private final long windowSize;
    +    private final SegmentedBytesStore.KeySchema keySchema = new WindowKeySchema();
     
         private String name;
         private ThreadCache cache;
    @@ -149,9 +150,16 @@ public synchronized WindowStoreIterator fetch(final K key, final long timeFro
             Bytes fromBytes = WindowStoreUtils.toBinaryKey(key, timeFrom, 0, serdes);
             Bytes toBytes = WindowStoreUtils.toBinaryKey(key, timeTo, 0, serdes);
     
    -        final WindowStoreIterator underlyingIterator = underlying.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo);
    +        final Bytes keyBytes = Bytes.wrap(serdes.rawKey(key));
    +        final WindowStoreIterator underlyingIterator = underlying.fetch(keyBytes, timeFrom, timeTo);
             final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(name, fromBytes, toBytes);
    -        return new MergedSortedCacheWindowStoreIterator<>(cacheIterator,
    +
    +        final HasNextCondition hasNextCondition = keySchema.hasNextCondition(keyBytes,
    +                                                                             timeFrom,
    +                                                                             timeTo);
    +        final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition);
    +
    +        return new MergedSortedCacheWindowStoreIterator<>(filteredCacheIterator,
                                                               underlyingIterator,
                                                               new StateSerdes<>(serdes.stateName(), Serdes.Long(), serdes.valueSerde()));
         }
    diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java
    new file mode 100644
    index 0000000000000..19370b95c94a1
    --- /dev/null
    +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java
    @@ -0,0 +1,73 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.streams.state.internals;
    +
    +import org.apache.kafka.common.utils.Bytes;
    +import org.apache.kafka.streams.KeyValue;
    +
    +import java.util.NoSuchElementException;
    +
    +class FilteredCacheIterator implements PeekingKeyValueIterator {
    +    private final PeekingKeyValueIterator cacheIterator;
    +    private final HasNextCondition hasNextCondition;
    +
    +    FilteredCacheIterator(final PeekingKeyValueIterator cacheIterator,
    +                          final HasNextCondition hasNextCondition) {
    +        this.cacheIterator = cacheIterator;
    +        this.hasNextCondition = hasNextCondition;
    +    }
    +
    +    @Override
    +    public void close() {
    +        // no-op
    +    }
    +
    +    @Override
    +    public Bytes peekNextKey() {
    +        if (!hasNext()) {
    +            throw new NoSuchElementException();
    +        }
    +        return cacheIterator.peekNextKey();
    +    }
    +
    +    @Override
    +    public boolean hasNext() {
    +        return hasNextCondition.hasNext(cacheIterator);
    +    }
    +
    +    @Override
    +    public KeyValue next() {
    +        if (!hasNext()) {
    +            throw new NoSuchElementException();
    +        }
    +        return cacheIterator.next();
    +
    +    }
    +
    +    @Override
    +    public void remove() {
    +        throw new UnsupportedOperationException();
    +    }
    +
    +    @Override
    +    public KeyValue peekNext() {
    +        if (!hasNext()) {
    +            throw new NoSuchElementException();
    +        }
    +        return cacheIterator.peekNext();
    +    }
    +}
    diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java
    index 7ed598ed97eb7..0a89da77cded8 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowKeySchema.java
    @@ -24,12 +24,6 @@
     import java.util.List;
     
     class WindowKeySchema implements RocksDBSegmentedBytesStore.KeySchema {
    -    private static final HasNextCondition ITERATOR_HAS_NEXT = new HasNextCondition() {
    -        @Override
    -        public boolean hasNext(final KeyValueIterator iterator) {
    -            return iterator.hasNext();
    -        }
    -    };
         private final StateSerdes serdes = new StateSerdes<>("window-store-key-schema", Serdes.Bytes(), Serdes.ByteArray());
     
         @Override
    @@ -49,7 +43,21 @@ public long segmentTimestamp(final Bytes key) {
     
         @Override
         public HasNextCondition hasNextCondition(final Bytes binaryKey, final long from, final long to) {
    -        return ITERATOR_HAS_NEXT;
    +        return new HasNextCondition() {
    +            @Override
    +            public boolean hasNext(final KeyValueIterator iterator) {
    +                if (iterator.hasNext()) {
    +                    final Bytes bytes = iterator.peekNextKey();
    +                    final Bytes keyBytes = WindowStoreUtils.bytesKeyFromBinaryKey(bytes.get());
    +                    if (!keyBytes.equals(binaryKey)) {
    +                        return false;
    +                    }
    +                    final long time = WindowStoreUtils.timestampFromBinaryKey(bytes.get());
    +                    return time >= from && time <= to;
    +                }
    +                return false;
    +            }
    +        };
         }
     
         @Override
    diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
    index c7b684671b50d..297a88e6659f2 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java
    @@ -19,6 +19,7 @@
     import org.apache.kafka.common.metrics.Metrics;
     import org.apache.kafka.common.serialization.Serdes;
     import org.apache.kafka.common.utils.Bytes;
    +import org.apache.kafka.common.utils.Utils;
     import org.apache.kafka.streams.KeyValue;
     import org.apache.kafka.streams.errors.InvalidStateStoreException;
     import org.apache.kafka.streams.kstream.Windowed;
    @@ -30,12 +31,17 @@
     import org.apache.kafka.streams.state.WindowStoreIterator;
     import org.apache.kafka.test.MockProcessorContext;
     import org.apache.kafka.test.TestUtils;
    +import org.junit.After;
     import org.junit.Before;
     import org.junit.Test;
     
     import java.io.IOException;
    +import java.util.List;
     
     import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize;
    +import static org.apache.kafka.test.StreamsTestUtils.toList;
    +import static org.hamcrest.CoreMatchers.equalTo;
    +import static org.hamcrest.MatcherAssert.assertThat;
     import static org.junit.Assert.assertArrayEquals;
     import static org.junit.Assert.assertEquals;
     import static org.junit.Assert.assertFalse;
    @@ -53,13 +59,12 @@ public class CachingWindowStoreTest {
         private ThreadCache cache;
         private String topic;
         private WindowKeySchema keySchema;
    -    private RocksDBWindowStore windowStore;
     
         @Before
         public void setUp() throws Exception {
             keySchema = new WindowKeySchema();
             underlying = new RocksDBSegmentedBytesStore("test", 30000, 3, keySchema);
    -        windowStore = new RocksDBWindowStore<>(underlying, Serdes.Bytes(), Serdes.ByteArray(), false);
    +        final RocksDBWindowStore windowStore = new RocksDBWindowStore<>(underlying, Serdes.Bytes(), Serdes.ByteArray(), false);
             cacheListener = new CachingKeyValueStoreTest.CacheFlushListenerStub<>();
             cachingStore = new CachingWindowStore<>(windowStore,
                                                     Serdes.String(),
    @@ -73,6 +78,10 @@ public void setUp() throws Exception {
             cachingStore.init(context, cachingStore);
         }
     
    +    @After
    +    public void closeStore() {
    +        cachingStore.close();
    +    }
     
         @Test
         public void shouldPutFetchFromCache() throws Exception {
    @@ -179,6 +188,18 @@ public void shouldThrowIfTryingToWriteToClosedCachingStore() throws Exception {
             cachingStore.put("a", "a");
         }
     
    +    @SuppressWarnings("unchecked")
    +    @Test
    +    public void shouldFetchAndIterateOverExactKeys() throws Exception {
    +        cachingStore.put("a", "0001", 0);
    +        cachingStore.put("aa", "0002", 0);
    +        cachingStore.put("a", "0003", 1);
    +        cachingStore.put("aa", "0004", 1);
    +        cachingStore.put("a", "0005", 60000);
    +
    +        final List> expected = Utils.mkList(KeyValue.pair(0L, "0001"), KeyValue.pair(1L, "0003"), KeyValue.pair(60000L, "0005"));
    +        assertThat(toList(cachingStore.fetch("a", 0, Long.MAX_VALUE)), equalTo(expected));
    +    }
     
         private int addItemsToCache() throws IOException {
             int cachedSize = 0;
    diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java
    new file mode 100644
    index 0000000000000..acded8cb51652
    --- /dev/null
    +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.kafka.streams.state.internals;
    +
    +import org.apache.kafka.common.utils.Bytes;
    +import org.apache.kafka.common.utils.Utils;
    +import org.apache.kafka.streams.KeyValue;
    +import org.apache.kafka.streams.state.KeyValueIterator;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import java.util.List;
    +
    +import static org.apache.kafka.test.StreamsTestUtils.toList;
    +import static org.hamcrest.CoreMatchers.equalTo;
    +import static org.hamcrest.MatcherAssert.assertThat;
    +import static org.junit.Assert.assertFalse;
    +import static org.junit.Assert.assertTrue;
    +
    +public class FilteredCacheIteratorTest {
    +
    +    @SuppressWarnings("unchecked")
    +    private final InMemoryKeyValueStore store = new InMemoryKeyValueStore("name", null, null);
    +    private final KeyValue firstEntry = KeyValue.pair(Bytes.wrap("a".getBytes()),
    +                                                                            new LRUCacheEntry("1".getBytes()));
    +    private final List> entries = Utils.mkList(
    +            firstEntry,
    +            KeyValue.pair(Bytes.wrap("b".getBytes()),
    +                          new LRUCacheEntry("2".getBytes())),
    +            KeyValue.pair(Bytes.wrap("c".getBytes()),
    +                          new LRUCacheEntry("3".getBytes())));
    +
    +    private FilteredCacheIterator allIterator;
    +    private FilteredCacheIterator firstEntryIterator;
    +
    +    @Before
    +    public void before() {
    +        store.putAll(entries);
    +        final HasNextCondition allCondition = new HasNextCondition() {
    +            @Override
    +            public boolean hasNext(final KeyValueIterator iterator) {
    +                return iterator.hasNext();
    +            }
    +        };
    +        allIterator = new FilteredCacheIterator(
    +                new DelegatingPeekingKeyValueIterator<>("",
    +                                                        store.all()), allCondition);
    +
    +        final HasNextCondition firstEntryCondition = new HasNextCondition() {
    +            @Override
    +            public boolean hasNext(final KeyValueIterator iterator) {
    +                return iterator.hasNext() && iterator.peekNextKey().equals(firstEntry.key);
    +            }
    +        };
    +        firstEntryIterator = new FilteredCacheIterator(
    +                new DelegatingPeekingKeyValueIterator<>("",
    +                                                        store.all()), firstEntryCondition);
    +
    +    }
    +
    +    @Test
    +    public void shouldAllowEntryMatchingHasNextCondition() throws Exception {
    +        final List> keyValues = toList(allIterator);
    +        assertThat(keyValues, equalTo(entries));
    +    }
    +
    +    @Test
    +    public void shouldPeekNextKey() throws Exception {
    +        while (allIterator.hasNext()) {
    +            final Bytes nextKey = allIterator.peekNextKey();
    +            final KeyValue next = allIterator.next();
    +            assertThat(next.key, equalTo(nextKey));
    +        }
    +    }
    +
    +    @Test
    +    public void shouldPeekNext() throws Exception {
    +        while (allIterator.hasNext()) {
    +            final KeyValue peeked = allIterator.peekNext();
    +            final KeyValue next = allIterator.next();
    +            assertThat(peeked, equalTo(next));
    +        }
    +    }
    +
    +    @Test
    +    public void shouldNotHaveNextIfHasNextConditionNotMet() throws Exception {
    +        assertTrue(firstEntryIterator.hasNext());
    +        firstEntryIterator.next();
    +        assertFalse(firstEntryIterator.hasNext());
    +    }
    +
    +    @Test
    +    public void shouldFilterEntriesNotMatchingHasNextCondition() throws Exception {
    +        final List> keyValues = toList(firstEntryIterator);
    +        assertThat(keyValues, equalTo(Utils.mkList(firstEntry)));
    +    }
    +
    +    @Test(expected = UnsupportedOperationException.class)
    +    public void shouldThrowUnsupportedOperationExeceptionOnRemove() throws Exception {
    +        allIterator.remove();
    +    }
    +
    +}
    \ No newline at end of file
    diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
    index b7dd94257f497..7352673c3de78 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java
    @@ -34,6 +34,7 @@
     import org.apache.kafka.streams.state.WindowStoreIterator;
     import org.apache.kafka.test.MockProcessorContext;
     import org.apache.kafka.test.TestUtils;
    +import org.junit.After;
     import org.junit.Test;
     
     import java.io.File;
    @@ -47,6 +48,8 @@
     import java.util.Map;
     import java.util.Set;
     
    +import static org.hamcrest.CoreMatchers.equalTo;
    +import static org.hamcrest.MatcherAssert.assertThat;
     import static org.junit.Assert.assertEquals;
     import static org.junit.Assert.assertFalse;
     import static org.junit.Assert.assertNull;
    @@ -87,6 +90,7 @@ public  void send(final String topic,
     
         private final File baseDir = TestUtils.tempDirectory("test");
         private final MockProcessorContext context = new MockProcessorContext(baseDir, Serdes.ByteArray(), Serdes.ByteArray(), recordCollector, cache);
    +    private WindowStore windowStore;
     
         @SuppressWarnings("unchecked")
         private  WindowStore createWindowStore(ProcessorContext context, final boolean enableCaching, final boolean retainDuplicates) {
    @@ -96,9 +100,15 @@ private  WindowStore createWindowStore(ProcessorContext context, fin
             return store;
         }
     
    +    @After
    +    public void closeStore() {
    +        windowStore.close();
    +    }
    +
    +    @SuppressWarnings("unchecked")
         @Test
         public void shouldOnlyIterateOpenSegments() throws Exception {
    -        final WindowStore windowStore = createWindowStore(context, false, true);
    +        windowStore = createWindowStore(context, false, true);
             long currentTime = 0;
             context.setRecordContext(createRecordContext(currentTime));
             windowStore.put(1, "one");
    @@ -128,494 +138,461 @@ private ProcessorRecordContext createRecordContext(final long time) {
             return new ProcessorRecordContext(time, 0, 0, "topic");
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testPutAndFetch() throws IOException {
    -        WindowStore store = createWindowStore(context, false, true);
    -        try {
    -            long startTime = segmentSize - 4L;
    -
    -            putFirstBatch(store, startTime, context);
    -
    -            assertEquals(Utils.mkList("zero"), toList(store.fetch(0, startTime + 0L - windowSize, startTime + 0L + windowSize)));
    -            assertEquals(Utils.mkList("one"), toList(store.fetch(1, startTime + 1L - windowSize, startTime + 1L + windowSize)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(3, startTime + 3L - windowSize, startTime + 3L + windowSize)));
    -            assertEquals(Utils.mkList("four"), toList(store.fetch(4, startTime + 4L - windowSize, startTime + 4L + windowSize)));
    -            assertEquals(Utils.mkList("five"), toList(store.fetch(5, startTime + 5L - windowSize, startTime + 5L + windowSize)));
    -
    -            putSecondBatch(store, startTime, context);
    -
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime - 2L - windowSize, startTime - 2L + windowSize)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime - 1L - windowSize, startTime - 1L + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1"), toList(store.fetch(2, startTime - windowSize, startTime + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(store.fetch(2, startTime + 1L - windowSize, startTime + 1L + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(store.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4"), toList(store.fetch(2, startTime + 3L - windowSize, startTime + 3L + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(store.fetch(2, startTime + 4L - windowSize, startTime + 4L + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 5L - windowSize, startTime + 5L + windowSize)));
    -            assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 6L - windowSize, startTime + 6L + windowSize)));
    -            assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 7L - windowSize, startTime + 7L + windowSize)));
    -            assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 8L - windowSize, startTime + 8L + windowSize)));
    -            assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 9L - windowSize, startTime + 9L + windowSize)));
    -            assertEquals(Utils.mkList("two+5", "two+6"), toList(store.fetch(2, startTime + 10L - windowSize, startTime + 10L + windowSize)));
    -            assertEquals(Utils.mkList("two+6"), toList(store.fetch(2, startTime + 11L - windowSize, startTime + 11L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 12L - windowSize, startTime + 12L + windowSize)));
    -
    -            // Flush the store and verify all current entries were properly flushed ...
    -            store.flush();
    -
    -            Map> entriesByKey = entriesByKey(changeLog, startTime);
    -
    -            assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
    -            assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
    -            assertEquals(Utils.mkSet("two@2", "two+1@3", "two+2@4", "two+3@5", "two+4@6", "two+5@7", "two+6@8"), entriesByKey.get(2));
    -            assertNull(entriesByKey.get(3));
    -            assertEquals(Utils.mkSet("four@4"), entriesByKey.get(4));
    -            assertEquals(Utils.mkSet("five@5"), entriesByKey.get(5));
    -            assertNull(entriesByKey.get(6));
    -
    -        } finally {
    -            store.close();
    -        }
    +        windowStore = createWindowStore(context, false, true);
    +        long startTime = segmentSize - 4L;
    +
    +        putFirstBatch(windowStore, startTime, context);
    +
    +        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L + windowSize)));
    +        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L - windowSize, startTime + 1L + windowSize)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L + windowSize)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L + windowSize)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L - windowSize, startTime + 5L + windowSize)));
    +
    +        putSecondBatch(windowStore, startTime, context);
    +
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L - windowSize, startTime - 2L + windowSize)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 1L - windowSize, startTime + 1L + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 3L - windowSize, startTime + 3L + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 4L - windowSize, startTime + 4L + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 5L - windowSize, startTime + 5L + windowSize)));
    +        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 6L - windowSize, startTime + 6L + windowSize)));
    +        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 7L - windowSize, startTime + 7L + windowSize)));
    +        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 8L - windowSize, startTime + 8L + windowSize)));
    +        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 9L - windowSize, startTime + 9L + windowSize)));
    +        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 10L - windowSize, startTime + 10L + windowSize)));
    +        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 11L - windowSize, startTime + 11L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L - windowSize, startTime + 12L + windowSize)));
    +
    +        // Flush the store and verify all current entries were properly flushed ...
    +        windowStore.flush();
    +
    +        Map> entriesByKey = entriesByKey(changeLog, startTime);
    +
    +        assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
    +        assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
    +        assertEquals(Utils.mkSet("two@2", "two+1@3", "two+2@4", "two+3@5", "two+4@6", "two+5@7", "two+6@8"), entriesByKey.get(2));
    +        assertNull(entriesByKey.get(3));
    +        assertEquals(Utils.mkSet("four@4"), entriesByKey.get(4));
    +        assertEquals(Utils.mkSet("five@5"), entriesByKey.get(5));
    +        assertNull(entriesByKey.get(6));
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testPutAndFetchBefore() throws IOException {
    -        WindowStore store = createWindowStore(context, false, true);
    -        try {
    -            long startTime = segmentSize - 4L;
    -
    -            putFirstBatch(store, startTime, context);
    -
    -            assertEquals(Utils.mkList("zero"), toList(store.fetch(0, startTime + 0L - windowSize, startTime + 0L)));
    -            assertEquals(Utils.mkList("one"), toList(store.fetch(1, startTime + 1L - windowSize, startTime + 1L)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime + 2L - windowSize, startTime + 2L)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(3, startTime + 3L - windowSize, startTime + 3L)));
    -            assertEquals(Utils.mkList("four"), toList(store.fetch(4, startTime + 4L - windowSize, startTime + 4L)));
    -            assertEquals(Utils.mkList("five"), toList(store.fetch(5, startTime + 5L - windowSize, startTime + 5L)));
    -
    -            putSecondBatch(store, startTime, context);
    -
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime - 1L - windowSize, startTime - 1L)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 0L - windowSize, startTime + 0L)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 1L - windowSize, startTime + 1L)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime + 2L - windowSize, startTime + 2L)));
    -            assertEquals(Utils.mkList("two", "two+1"), toList(store.fetch(2, startTime + 3L - windowSize, startTime + 3L)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(store.fetch(2, startTime + 4L - windowSize, startTime + 4L)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(store.fetch(2, startTime + 5L - windowSize, startTime + 5L)));
    -            assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(store.fetch(2, startTime + 6L - windowSize, startTime + 6L)));
    -            assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(store.fetch(2, startTime + 7L - windowSize, startTime + 7L)));
    -            assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 8L - windowSize, startTime + 8L)));
    -            assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 9L - windowSize, startTime + 9L)));
    -            assertEquals(Utils.mkList("two+5", "two+6"), toList(store.fetch(2, startTime + 10L - windowSize, startTime + 10L)));
    -            assertEquals(Utils.mkList("two+6"), toList(store.fetch(2, startTime + 11L - windowSize, startTime + 11L)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 12L - windowSize, startTime + 12L)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 13L - windowSize, startTime + 13L)));
    -
    -            // Flush the store and verify all current entries were properly flushed ...
    -            store.flush();
    -
    -            Map> entriesByKey = entriesByKey(changeLog, startTime);
    -
    -            assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
    -            assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
    -            assertEquals(Utils.mkSet("two@2", "two+1@3", "two+2@4", "two+3@5", "two+4@6", "two+5@7", "two+6@8"), entriesByKey.get(2));
    -            assertNull(entriesByKey.get(3));
    -            assertEquals(Utils.mkSet("four@4"), entriesByKey.get(4));
    -            assertEquals(Utils.mkSet("five@5"), entriesByKey.get(5));
    -            assertNull(entriesByKey.get(6));
    -
    -        } finally {
    -            store.close();
    -        }
    +        windowStore = createWindowStore(context, false, true);
    +        long startTime = segmentSize - 4L;
    +
    +        putFirstBatch(windowStore, startTime, context);
    +
    +        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L - windowSize, startTime + 0L)));
    +        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L - windowSize, startTime + 1L)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L - windowSize, startTime + 3L)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L - windowSize, startTime + 4L)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L - windowSize, startTime + 5L)));
    +
    +        putSecondBatch(windowStore, startTime, context);
    +
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 1L - windowSize, startTime - 1L)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 0L - windowSize, startTime + 0L)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 1L - windowSize, startTime + 1L)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L - windowSize, startTime + 2L)));
    +        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime + 3L - windowSize, startTime + 3L)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 4L - windowSize, startTime + 4L)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 5L - windowSize, startTime + 5L)));
    +        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 6L - windowSize, startTime + 6L)));
    +        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 7L - windowSize, startTime + 7L)));
    +        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 8L - windowSize, startTime + 8L)));
    +        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 9L - windowSize, startTime + 9L)));
    +        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 10L - windowSize, startTime + 10L)));
    +        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 11L - windowSize, startTime + 11L)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L - windowSize, startTime + 12L)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 13L - windowSize, startTime + 13L)));
    +
    +        // Flush the store and verify all current entries were properly flushed ...
    +        windowStore.flush();
    +
    +        Map> entriesByKey = entriesByKey(changeLog, startTime);
    +
    +        assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
    +        assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
    +        assertEquals(Utils.mkSet("two@2", "two+1@3", "two+2@4", "two+3@5", "two+4@6", "two+5@7", "two+6@8"), entriesByKey.get(2));
    +        assertNull(entriesByKey.get(3));
    +        assertEquals(Utils.mkSet("four@4"), entriesByKey.get(4));
    +        assertEquals(Utils.mkSet("five@5"), entriesByKey.get(5));
    +        assertNull(entriesByKey.get(6));
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testPutAndFetchAfter() throws IOException {
    -        WindowStore store = createWindowStore(context, false, true);
    -        try {
    -            long startTime = segmentSize - 4L;
    -
    -            putFirstBatch(store, startTime, context);
    -
    -            assertEquals(Utils.mkList("zero"), toList(store.fetch(0, startTime + 0L, startTime + 0L + windowSize)));
    -            assertEquals(Utils.mkList("one"), toList(store.fetch(1, startTime + 1L, startTime + 1L + windowSize)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime + 2L, startTime + 2L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(3, startTime + 3L, startTime + 3L + windowSize)));
    -            assertEquals(Utils.mkList("four"), toList(store.fetch(4, startTime + 4L, startTime + 4L + windowSize)));
    -            assertEquals(Utils.mkList("five"), toList(store.fetch(5, startTime + 5L, startTime + 5L + windowSize)));
    -
    -            putSecondBatch(store, startTime, context);
    -
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime - 2L, startTime - 2L + windowSize)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime - 1L, startTime - 1L + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1"), toList(store.fetch(2, startTime, startTime + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(store.fetch(2, startTime + 1L, startTime + 1L + windowSize)));
    -            assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(store.fetch(2, startTime + 2L, startTime + 2L + windowSize)));
    -            assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(store.fetch(2, startTime + 3L, startTime + 3L + windowSize)));
    -            assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(store.fetch(2, startTime + 4L, startTime + 4L + windowSize)));
    -            assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 5L, startTime + 5L + windowSize)));
    -            assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(store.fetch(2, startTime + 6L, startTime + 6L + windowSize)));
    -            assertEquals(Utils.mkList("two+5", "two+6"), toList(store.fetch(2, startTime + 7L, startTime + 7L + windowSize)));
    -            assertEquals(Utils.mkList("two+6"), toList(store.fetch(2, startTime + 8L, startTime + 8L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 9L, startTime + 9L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 10L, startTime + 10L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 11L, startTime + 11L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + 12L, startTime + 12L + windowSize)));
    -
    -            // Flush the store and verify all current entries were properly flushed ...
    -            store.flush();
    -
    -            Map> entriesByKey = entriesByKey(changeLog, startTime);
    -
    -            assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
    -            assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
    -            assertEquals(Utils.mkSet("two@2", "two+1@3", "two+2@4", "two+3@5", "two+4@6", "two+5@7", "two+6@8"), entriesByKey.get(2));
    -            assertNull(entriesByKey.get(3));
    -            assertEquals(Utils.mkSet("four@4"), entriesByKey.get(4));
    -            assertEquals(Utils.mkSet("five@5"), entriesByKey.get(5));
    -            assertNull(entriesByKey.get(6));
    -
    -        } finally {
    -            store.close();
    -        }
    +        windowStore = createWindowStore(context, false, true);
    +        long startTime = segmentSize - 4L;
    +
    +        putFirstBatch(windowStore, startTime, context);
    +
    +        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime + 0L, startTime + 0L + windowSize)));
    +        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + 1L, startTime + 1L + windowSize)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + 2L, startTime + 2L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + 3L, startTime + 3L + windowSize)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + 4L, startTime + 4L + windowSize)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + 5L, startTime + 5L + windowSize)));
    +
    +        putSecondBatch(windowStore, startTime, context);
    +
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime - 2L, startTime - 2L + windowSize)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime - 1L, startTime - 1L + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, startTime, startTime + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, startTime + 1L, startTime + 1L + windowSize)));
    +        assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, startTime + 2L, startTime + 2L + windowSize)));
    +        assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, startTime + 3L, startTime + 3L + windowSize)));
    +        assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, startTime + 4L, startTime + 4L + windowSize)));
    +        assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 5L, startTime + 5L + windowSize)));
    +        assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, startTime + 6L, startTime + 6L + windowSize)));
    +        assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, startTime + 7L, startTime + 7L + windowSize)));
    +        assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, startTime + 8L, startTime + 8L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 9L, startTime + 9L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 10L, startTime + 10L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 11L, startTime + 11L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + 12L, startTime + 12L + windowSize)));
    +
    +        // Flush the store and verify all current entries were properly flushed ...
    +        windowStore.flush();
    +
    +        Map> entriesByKey = entriesByKey(changeLog, startTime);
    +
    +        assertEquals(Utils.mkSet("zero@0"), entriesByKey.get(0));
    +        assertEquals(Utils.mkSet("one@1"), entriesByKey.get(1));
    +        assertEquals(Utils.mkSet("two@2", "two+1@3", "two+2@4", "two+3@5", "two+4@6", "two+5@7", "two+6@8"), entriesByKey.get(2));
    +        assertNull(entriesByKey.get(3));
    +        assertEquals(Utils.mkSet("four@4"), entriesByKey.get(4));
    +        assertEquals(Utils.mkSet("five@5"), entriesByKey.get(5));
    +        assertNull(entriesByKey.get(6));
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testPutSameKeyTimestamp() throws IOException {
    -        WindowStore store = createWindowStore(context, false, true);
    -        try {
    -            long startTime = segmentSize - 4L;
    -
    -            context.setRecordContext(createRecordContext(startTime));
    -            store.put(0, "zero");
    +        windowStore = createWindowStore(context, false, true);
    +        long startTime = segmentSize - 4L;
     
    -            assertEquals(Utils.mkList("zero"), toList(store.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        context.setRecordContext(createRecordContext(startTime));
    +        windowStore.put(0, "zero");
     
    -            store.put(0, "zero");
    -            store.put(0, "zero+");
    -            store.put(0, "zero++");
    +        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
     
    -            assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(store.fetch(0, startTime - windowSize, startTime + windowSize)));
    -            assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(store.fetch(0, startTime + 1L - windowSize, startTime + 1L + windowSize)));
    -            assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(store.fetch(0, startTime + 2L - windowSize, startTime + 2L + windowSize)));
    -            assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(store.fetch(0, startTime + 3L - windowSize, startTime + 3L + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(0, startTime + 4L - windowSize, startTime + 4L + windowSize)));
    +        windowStore.put(0, "zero");
    +        windowStore.put(0, "zero+");
    +        windowStore.put(0, "zero++");
     
    -            // Flush the store and verify all current entries were properly flushed ...
    -            store.flush();
    +        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 1L - windowSize, startTime + 1L + windowSize)));
    +        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 2L - windowSize, startTime + 2L + windowSize)));
    +        assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, startTime + 3L - windowSize, startTime + 3L + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime + 4L - windowSize, startTime + 4L + windowSize)));
     
    -            Map> entriesByKey = entriesByKey(changeLog, startTime);
    +        // Flush the store and verify all current entries were properly flushed ...
    +        windowStore.flush();
     
    -            assertEquals(Utils.mkSet("zero@0", "zero@0", "zero+@0", "zero++@0"), entriesByKey.get(0));
    +        Map> entriesByKey = entriesByKey(changeLog, startTime);
     
    -        } finally {
    -            store.close();
    -        }
    +        assertEquals(Utils.mkSet("zero@0", "zero@0", "zero+@0", "zero++@0"), entriesByKey.get(0));
         }
     
         @Test
         public void testCachingEnabled() throws IOException {
    -        WindowStore store = createWindowStore(context, true, false);
    -        assertTrue(store instanceof CachedStateStore);
    +        windowStore = createWindowStore(context, true, false);
    +        assertTrue(windowStore instanceof CachedStateStore);
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testRolling() throws IOException {
    -        WindowStore store = createWindowStore(context, false, true);
    +        windowStore = createWindowStore(context, false, true);
    +
    +        // to validate segments
    +        final Segments segments = new Segments(windowName, retentionPeriod, numSegments);
    +        long startTime = segmentSize * 2;
    +        long incr = segmentSize / 2;
    +        context.setRecordContext(createRecordContext(startTime));
    +        windowStore.put(0, "zero");
    +        assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
    +
    +        context.setRecordContext(createRecordContext(startTime + incr));
    +        windowStore.put(1, "one");
    +        assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
    +
    +        context.setRecordContext(createRecordContext(startTime + incr * 2));
    +        windowStore.put(2, "two");
    +        assertEquals(Utils.mkSet(segments.segmentName(2),
    +                                 segments.segmentName(3)), segmentDirs(baseDir));
    +
    +        context.setRecordContext(createRecordContext(startTime + incr * 4));
    +        windowStore.put(4, "four");
    +        assertEquals(Utils.mkSet(segments.segmentName(2),
    +                                 segments.segmentName(3),
    +                                 segments.segmentName(4)), segmentDirs(baseDir));
    +
    +
    +        context.setRecordContext(createRecordContext(startTime + incr * 5));
    +        windowStore.put(5, "five");
    +        assertEquals(Utils.mkSet(segments.segmentName(2),
    +                                 segments.segmentName(3),
    +                                 segments.segmentName(4)), segmentDirs(baseDir));
    +
    +        assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    +
    +        context.setRecordContext(createRecordContext(startTime + incr * 6));
    +        windowStore.put(6, "six");
    +        assertEquals(Utils.mkSet(segments.segmentName(3),
    +                                 segments.segmentName(4),
    +                                 segments.segmentName(5)), segmentDirs(baseDir));
    +
    +
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    +        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    +
    +
    +        context.setRecordContext(createRecordContext(startTime + incr * 7));
    +        windowStore.put(7, "seven");
    +        assertEquals(Utils.mkSet(segments.segmentName(3),
    +                                 segments.segmentName(4),
    +                                 segments.segmentName(5)), segmentDirs(baseDir));
    +
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    +        assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    +        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    +        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    +
    +        context.setRecordContext(createRecordContext(startTime + incr * 8));
    +        windowStore.put(8, "eight");
    +        assertEquals(Utils.mkSet(segments.segmentName(4),
    +                                 segments.segmentName(5),
    +                                 segments.segmentName(6)), segmentDirs(baseDir));
    +
    +
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    +        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    +        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    +        assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
    +
    +        // check segment directories
    +        windowStore.flush();
    +        assertEquals(Utils.mkSet(segments.segmentName(4),
    +                                 segments.segmentName(5),
    +                                 segments.segmentName(6)), segmentDirs(baseDir));
    +
     
    -        try {
    -            // to validate segments
    -            final Segments segments = new Segments(windowName, retentionPeriod, numSegments);
    -            long startTime = segmentSize * 2;
    -            long incr = segmentSize / 2;
    -            context.setRecordContext(createRecordContext(startTime));
    -            store.put(0, "zero");
    -            assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
    -
    -            context.setRecordContext(createRecordContext(startTime + incr));
    -            store.put(1, "one");
    -            assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir));
    -
    -            context.setRecordContext(createRecordContext(startTime + incr * 2));
    -            store.put(2, "two");
    -            assertEquals(Utils.mkSet(segments.segmentName(2),
    -                    segments.segmentName(3)), segmentDirs(baseDir));
    -
    -            context.setRecordContext(createRecordContext(startTime + incr * 4));
    -            store.put(4, "four");
    -            assertEquals(Utils.mkSet(segments.segmentName(2),
    -                    segments.segmentName(3),
    -                    segments.segmentName(4)), segmentDirs(baseDir));
    -
    -
    -            context.setRecordContext(createRecordContext(startTime + incr * 5));
    -            store.put(5, "five");
    -            assertEquals(Utils.mkSet(segments.segmentName(2),
    -                    segments.segmentName(3),
    -                    segments.segmentName(4)), segmentDirs(baseDir));
    -
    -            assertEquals(Utils.mkList("zero"), toList(store.fetch(0, startTime - windowSize, startTime + windowSize)));
    -            assertEquals(Utils.mkList("one"), toList(store.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    -            assertEquals(Utils.mkList("four"), toList(store.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    -            assertEquals(Utils.mkList("five"), toList(store.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    -
    -            context.setRecordContext(createRecordContext(startTime + incr * 6));
    -            store.put(6, "six");
    -            assertEquals(Utils.mkSet(segments.segmentName(3),
    -                    segments.segmentName(4),
    -                    segments.segmentName(5)), segmentDirs(baseDir));
    -
    -
    -            assertEquals(Utils.mkList(), toList(store.fetch(0, startTime - windowSize, startTime + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    -            assertEquals(Utils.mkList("four"), toList(store.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    -            assertEquals(Utils.mkList("five"), toList(store.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    -            assertEquals(Utils.mkList("six"), toList(store.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    -
    -
    -            context.setRecordContext(createRecordContext(startTime + incr * 7));
    -            store.put(7, "seven");
    -            assertEquals(Utils.mkSet(segments.segmentName(3),
    -                    segments.segmentName(4),
    -                    segments.segmentName(5)), segmentDirs(baseDir));
    -
    -            assertEquals(Utils.mkList(), toList(store.fetch(0, startTime - windowSize, startTime + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    -            assertEquals(Utils.mkList("two"), toList(store.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    -            assertEquals(Utils.mkList("four"), toList(store.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    -            assertEquals(Utils.mkList("five"), toList(store.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    -            assertEquals(Utils.mkList("six"), toList(store.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    -            assertEquals(Utils.mkList("seven"), toList(store.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    -
    -            context.setRecordContext(createRecordContext(startTime + incr * 8));
    -            store.put(8, "eight");
    -            assertEquals(Utils.mkSet(segments.segmentName(4),
    -                    segments.segmentName(5),
    -                    segments.segmentName(6)), segmentDirs(baseDir));
    -
    -
    -            assertEquals(Utils.mkList(), toList(store.fetch(0, startTime - windowSize, startTime + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    -            assertEquals(Utils.mkList("four"), toList(store.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    -            assertEquals(Utils.mkList("five"), toList(store.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    -            assertEquals(Utils.mkList("six"), toList(store.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    -            assertEquals(Utils.mkList("seven"), toList(store.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    -            assertEquals(Utils.mkList("eight"), toList(store.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
    -
    -            // check segment directories
    -            store.flush();
    -            assertEquals(Utils.mkSet(segments.segmentName(4),
    -                    segments.segmentName(5),
    -                    segments.segmentName(6)), segmentDirs(baseDir));
    -
    -        } finally {
    -            store.close();
    -        }
         }
     
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testRestore() throws IOException {
             long startTime = segmentSize * 2;
             long incr = segmentSize / 2;
     
    -        WindowStore store = createWindowStore(context, false, true);
    -        try {
    -            context.setRecordContext(createRecordContext(startTime));
    -            store.put(0, "zero");
    -            context.setRecordContext(createRecordContext(startTime + incr));
    -            store.put(1, "one");
    -            context.setRecordContext(createRecordContext(startTime + incr * 2));
    -            store.put(2, "two");
    -            context.setRecordContext(createRecordContext(startTime + incr * 3));
    -            store.put(3, "three");
    -            context.setRecordContext(createRecordContext(startTime + incr * 4));
    -            store.put(4, "four");
    -            context.setRecordContext(createRecordContext(startTime + incr * 5));
    -            store.put(5, "five");
    -            context.setRecordContext(createRecordContext(startTime + incr * 6));
    -            store.put(6, "six");
    -            context.setRecordContext(createRecordContext(startTime + incr * 7));
    -            store.put(7, "seven");
    -            context.setRecordContext(createRecordContext(startTime + incr * 8));
    -            store.put(8, "eight");
    -            store.flush();
    -
    -        } finally {
    -            store.close();
    -        }
    +        windowStore = createWindowStore(context, false, true);
    +        context.setRecordContext(createRecordContext(startTime));
    +        windowStore.put(0, "zero");
    +        context.setRecordContext(createRecordContext(startTime + incr));
    +        windowStore.put(1, "one");
    +        context.setRecordContext(createRecordContext(startTime + incr * 2));
    +        windowStore.put(2, "two");
    +        context.setRecordContext(createRecordContext(startTime + incr * 3));
    +        windowStore.put(3, "three");
    +        context.setRecordContext(createRecordContext(startTime + incr * 4));
    +        windowStore.put(4, "four");
    +        context.setRecordContext(createRecordContext(startTime + incr * 5));
    +        windowStore.put(5, "five");
    +        context.setRecordContext(createRecordContext(startTime + incr * 6));
    +        windowStore.put(6, "six");
    +        context.setRecordContext(createRecordContext(startTime + incr * 7));
    +        windowStore.put(7, "seven");
    +        context.setRecordContext(createRecordContext(startTime + incr * 8));
    +        windowStore.put(8, "eight");
    +        windowStore.flush();
    +
    +        windowStore.close();
     
             // remove local store image
             Utils.delete(baseDir);
     
    -        WindowStore store2 = createWindowStore(context, false, true);
    -        assertEquals(Utils.mkList(), toList(store2.fetch(0, startTime - windowSize, startTime + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    -        assertEquals(Utils.mkList(), toList(store2.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
    -
    -        try {
    -            context.restore(windowName, changeLog);
    -
    -            assertEquals(Utils.mkList(), toList(store2.fetch(0, startTime - windowSize, startTime + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store2.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store2.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    -            assertEquals(Utils.mkList(), toList(store2.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    -            assertEquals(Utils.mkList("four"), toList(store2.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    -            assertEquals(Utils.mkList("five"), toList(store2.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    -            assertEquals(Utils.mkList("six"), toList(store2.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    -            assertEquals(Utils.mkList("seven"), toList(store2.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    -            assertEquals(Utils.mkList("eight"), toList(store2.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
    -
    -            // check segment directories
    -            store2.flush();
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
    -                    segmentDirs(baseDir)
    -            );
    -        } finally {
    -            store2.close();
    -        }
    +        windowStore = createWindowStore(context, false, true);
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
    +
    +        context.restore(windowName, changeLog);
    +
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(0, startTime - windowSize, startTime + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(1, startTime + incr - windowSize, startTime + incr + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(2, startTime + incr * 2 - windowSize, startTime + incr * 2 + windowSize)));
    +        assertEquals(Utils.mkList(), toList(windowStore.fetch(3, startTime + incr * 3 - windowSize, startTime + incr * 3 + windowSize)));
    +        assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, startTime + incr * 4 - windowSize, startTime + incr * 4 + windowSize)));
    +        assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, startTime + incr * 5 - windowSize, startTime + incr * 5 + windowSize)));
    +        assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, startTime + incr * 6 - windowSize, startTime + incr * 6 + windowSize)));
    +        assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, startTime + incr * 7 - windowSize, startTime + incr * 7 + windowSize)));
    +        assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, startTime + incr * 8 - windowSize, startTime + incr * 8 + windowSize)));
    +
    +        // check segment directories
    +        windowStore.flush();
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
    +                segmentDirs(baseDir)
    +        );
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testSegmentMaintenance() throws IOException {
    -        WindowStore store = createWindowStore(context, false, true);
    -        try {
    -            context.setTime(0L);
    -            context.setRecordContext(createRecordContext(0));
    -            store.put(0, "v");
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(0L)),
    -                    segmentDirs(baseDir)
    -            );
    -
    -            context.setRecordContext(createRecordContext(59999));
    -            store.put(0, "v");
    -            store.put(0, "v");
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(0L)),
    -                    segmentDirs(baseDir)
    -            );
    -
    -            context.setRecordContext(createRecordContext(60000));
    -            store.put(0, "v");
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
    -                    segmentDirs(baseDir)
    -            );
    -
    -            WindowStoreIterator iter;
    -            int fetchedCount;
    -
    -            iter = store.fetch(0, 0L, 240000L);
    -            fetchedCount = 0;
    -            while (iter.hasNext()) {
    -                iter.next();
    -                fetchedCount++;
    -            }
    -            assertEquals(4, fetchedCount);
    +        windowStore = createWindowStore(context, false, true);
    +        context.setTime(0L);
    +        context.setRecordContext(createRecordContext(0));
    +        windowStore.put(0, "v");
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(0L)),
    +                segmentDirs(baseDir)
    +        );
    +
    +        context.setRecordContext(createRecordContext(59999));
    +        windowStore.put(0, "v");
    +        windowStore.put(0, "v");
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(0L)),
    +                segmentDirs(baseDir)
    +        );
    +
    +        context.setRecordContext(createRecordContext(60000));
    +        windowStore.put(0, "v");
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
    +                segmentDirs(baseDir)
    +        );
    +
    +        WindowStoreIterator iter;
    +        int fetchedCount;
    +
    +        iter = windowStore.fetch(0, 0L, 240000L);
    +        fetchedCount = 0;
    +        while (iter.hasNext()) {
    +            iter.next();
    +            fetchedCount++;
    +        }
    +        assertEquals(4, fetchedCount);
     
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
    -                    segmentDirs(baseDir)
    -            );
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)),
    +                segmentDirs(baseDir)
    +        );
     
    -            context.setRecordContext(createRecordContext(180000));
    -            store.put(0, "v");
    +        context.setRecordContext(createRecordContext(180000));
    +        windowStore.put(0, "v");
     
    -            iter = store.fetch(0, 0L, 240000L);
    -            fetchedCount = 0;
    -            while (iter.hasNext()) {
    -                iter.next();
    -                fetchedCount++;
    -            }
    -            assertEquals(2, fetchedCount);
    +        iter = windowStore.fetch(0, 0L, 240000L);
    +        fetchedCount = 0;
    +        while (iter.hasNext()) {
    +            iter.next();
    +            fetchedCount++;
    +        }
    +        assertEquals(2, fetchedCount);
     
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(1L), segments.segmentName(3L)),
    -                    segmentDirs(baseDir)
    -            );
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(1L), segments.segmentName(3L)),
    +                segmentDirs(baseDir)
    +        );
     
    -            context.setRecordContext(createRecordContext(300000));
    -            store.put(0, "v");
    +        context.setRecordContext(createRecordContext(300000));
    +        windowStore.put(0, "v");
     
    -            iter = store.fetch(0, 240000L, 1000000L);
    -            fetchedCount = 0;
    -            while (iter.hasNext()) {
    -                iter.next();
    -                fetchedCount++;
    -            }
    -            assertEquals(1, fetchedCount);
    +        iter = windowStore.fetch(0, 240000L, 1000000L);
    +        fetchedCount = 0;
    +        while (iter.hasNext()) {
    +            iter.next();
    +            fetchedCount++;
    +        }
    +        assertEquals(1, fetchedCount);
     
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(3L), segments.segmentName(5L)),
    -                    segmentDirs(baseDir)
    -            );
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(3L), segments.segmentName(5L)),
    +                segmentDirs(baseDir)
    +        );
     
    -        } finally {
    -            store.close();
    -        }
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void testInitialLoading() throws IOException {
             File storeDir = new File(baseDir, windowName);
     
    -        WindowStore store = createWindowStore(context, false, true);
    +        windowStore = createWindowStore(context, false, true);
     
    -        try {
    -            new File(storeDir, segments.segmentName(0L)).mkdir();
    -            new File(storeDir, segments.segmentName(1L)).mkdir();
    -            new File(storeDir, segments.segmentName(2L)).mkdir();
    -            new File(storeDir, segments.segmentName(3L)).mkdir();
    -            new File(storeDir, segments.segmentName(4L)).mkdir();
    -            new File(storeDir, segments.segmentName(5L)).mkdir();
    -            new File(storeDir, segments.segmentName(6L)).mkdir();
    -        } finally {
    -            store.close();
    -        }
    +        new File(storeDir, segments.segmentName(0L)).mkdir();
    +        new File(storeDir, segments.segmentName(1L)).mkdir();
    +        new File(storeDir, segments.segmentName(2L)).mkdir();
    +        new File(storeDir, segments.segmentName(3L)).mkdir();
    +        new File(storeDir, segments.segmentName(4L)).mkdir();
    +        new File(storeDir, segments.segmentName(5L)).mkdir();
    +        new File(storeDir, segments.segmentName(6L)).mkdir();
    +        windowStore.close();
     
    -        store = createWindowStore(context, false, true);
    +        windowStore = createWindowStore(context, false, true);
     
    -        try {
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
    -                    segmentDirs(baseDir)
    -            );
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
    +                segmentDirs(baseDir)
    +        );
     
    -            try (WindowStoreIterator iter = store.fetch(0, 0L, 1000000L)) {
    -                while (iter.hasNext()) {
    -                    iter.next();
    -                }
    +        try (WindowStoreIterator iter = windowStore.fetch(0, 0L, 1000000L)) {
    +            while (iter.hasNext()) {
    +                iter.next();
                 }
    -
    -            assertEquals(
    -                    Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
    -                    segmentDirs(baseDir)
    -            );
    -
    -        } finally {
    -            store.close();
             }
    +
    +        assertEquals(
    +                Utils.mkSet(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)),
    +                segmentDirs(baseDir)
    +        );
         }
     
    +    @SuppressWarnings("unchecked")
         @Test
         public void shouldCloseOpenIteratorsWhenStoreIsClosedAndThrowInvalidStateStoreExceptionOnHasNextAndNext() throws Exception {
    -        final WindowStore windowStore = createWindowStore(context, false, true);
    +        windowStore = createWindowStore(context, false, true);
             context.setRecordContext(createRecordContext(0));
             windowStore.put(1, "one", 1L);
             windowStore.put(1, "two", 2L);
    @@ -639,6 +616,34 @@ public void shouldCloseOpenIteratorsWhenStoreIsClosedAndThrowInvalidStateStoreEx
             }
         }
     
    +    @SuppressWarnings("unchecked")
    +    @Test
    +    public void shouldFetchAndIterateOverExactKeys() throws Exception {
    +        final RocksDBWindowStoreSupplier supplier =
    +                new RocksDBWindowStoreSupplier<>(
    +                        "window",
    +                        60 * 1000L * 2, 3,
    +                        true,
    +                        Serdes.String(),
    +                        Serdes.String(),
    +                        windowSize,
    +                        true,
    +                        Collections.emptyMap(),
    +                        false);
    +
    +        windowStore = supplier.get();
    +        windowStore.init(context, windowStore);
    +
    +        windowStore.put("a", "0001", 0);
    +        windowStore.put("aa", "0002", 0);
    +        windowStore.put("a", "0003", 1);
    +        windowStore.put("aa", "0004", 1);
    +        windowStore.put("a", "0005", 60000);
    +
    +        final List expected = Utils.mkList("0001", "0003", "0005");
    +        assertThat(toList(windowStore.fetch("a", 0, Long.MAX_VALUE)), equalTo(expected));
    +    }
    +
         private void putFirstBatch(final WindowStore store, final long startTime, final MockProcessorContext context) {
             context.setRecordContext(createRecordContext(startTime));
             store.put(0, "zero");
    
    From 52a15d7c0b88da11409954321463b8b57b133a23 Mon Sep 17 00:00:00 2001
    From: Ewen Cheslack-Postava 
    Date: Tue, 14 Mar 2017 17:20:49 -0700
    Subject: [PATCH 095/101] KAFKA-4783: Add ByteArrayConverter (KIP-128)
    
    Author: Ewen Cheslack-Postava 
    
    Reviewers: Guozhang Wang 
    
    Closes #2599 from ewencp/kafka-4783-byte-array-converter
    ---
     checkstyle/import-control.xml                 |  4 +
     .../converters/ByteArrayConverter.java        | 52 +++++++++++
     .../converters/ByteArrayConverterTest.java    | 90 +++++++++++++++++++
     3 files changed, 146 insertions(+)
     create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java
     create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java
    
    diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
    index fa98593b0ff26..80747e14b528d 100644
    --- a/checkstyle/import-control.xml
    +++ b/checkstyle/import-control.xml
    @@ -221,6 +221,10 @@
           
         
     
    +    
    +      
    +    
    +    
         
           
           
    diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java
    new file mode 100644
    index 0000000000000..f592624cf2ffb
    --- /dev/null
    +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java
    @@ -0,0 +1,52 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + **/
    +
    +package org.apache.kafka.connect.converters;
    +
    +import org.apache.kafka.connect.data.Schema;
    +import org.apache.kafka.connect.data.SchemaAndValue;
    +import org.apache.kafka.connect.errors.DataException;
    +import org.apache.kafka.connect.storage.Converter;
    +
    +import java.util.Map;
    +
    +/**
    + * Pass-through converter for raw byte data.
    + */
    +public class ByteArrayConverter implements Converter {
    +
    +    @Override
    +    public void configure(Map configs, boolean isKey) {
    +    }
    +
    +    @Override
    +    public byte[] fromConnectData(String topic, Schema schema, Object value) {
    +        if (schema != null && schema.type() != Schema.Type.BYTES)
    +            throw new DataException("Invalid schema type for ByteArrayConverter: " + schema.type().toString());
    +
    +        if (value != null && !(value instanceof byte[]))
    +            throw new DataException("ByteArrayConverter is not compatible with objects of type " + value.getClass());
    +
    +        return (byte[]) value;
    +    }
    +
    +    @Override
    +    public SchemaAndValue toConnectData(String topic, byte[] value) {
    +        return new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value);
    +    }
    +
    +}
    \ No newline at end of file
    diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java
    new file mode 100644
    index 0000000000000..e0d533f762225
    --- /dev/null
    +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java
    @@ -0,0 +1,90 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + **/
    +
    +package org.apache.kafka.connect.converters;
    +
    +import org.apache.kafka.connect.data.Schema;
    +import org.apache.kafka.connect.data.SchemaAndValue;
    +import org.apache.kafka.connect.errors.DataException;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import java.nio.charset.StandardCharsets;
    +import java.util.Arrays;
    +import java.util.Collections;
    +
    +import static org.junit.Assert.assertArrayEquals;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +
    +public class ByteArrayConverterTest {
    +    private static final String TOPIC = "topic";
    +    private static final byte[] SAMPLE_BYTES = "sample string".getBytes(StandardCharsets.UTF_8);
    +
    +    private ByteArrayConverter converter = new ByteArrayConverter();
    +
    +    @Before
    +    public void setUp() {
    +        converter.configure(Collections.emptyMap(), false);
    +    }
    +
    +    @Test
    +    public void testFromConnect() {
    +        assertArrayEquals(
    +                SAMPLE_BYTES,
    +                converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, SAMPLE_BYTES)
    +        );
    +    }
    +
    +    @Test
    +    public void testFromConnectSchemaless() {
    +        assertArrayEquals(
    +                SAMPLE_BYTES,
    +                converter.fromConnectData(TOPIC, null, SAMPLE_BYTES)
    +        );
    +    }
    +
    +    @Test(expected = DataException.class)
    +    public void testFromConnectBadSchema() {
    +        converter.fromConnectData(TOPIC, Schema.INT32_SCHEMA, SAMPLE_BYTES);
    +    }
    +
    +    @Test(expected = DataException.class)
    +    public void testFromConnectInvalidValue() {
    +        converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, 12);
    +    }
    +
    +    @Test
    +    public void testFromConnectNull() {
    +        assertNull(converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, null));
    +    }
    +
    +    @Test
    +    public void testToConnect() {
    +        SchemaAndValue data = converter.toConnectData(TOPIC, SAMPLE_BYTES);
    +        assertEquals(Schema.OPTIONAL_BYTES_SCHEMA, data.schema());
    +        assertTrue(Arrays.equals(SAMPLE_BYTES, (byte[]) data.value()));
    +    }
    +
    +    @Test
    +    public void testToConnectNull() {
    +        SchemaAndValue data = converter.toConnectData(TOPIC, null);
    +        assertEquals(Schema.OPTIONAL_BYTES_SCHEMA, data.schema());
    +        assertNull(data.value());
    +    }
    +}
    
    From 962c378cc6cdb178541155af419c24e463516a5d Mon Sep 17 00:00:00 2001
    From: Ismael Juma 
    Date: Wed, 15 Mar 2017 14:46:17 +0000
    Subject: [PATCH 096/101] HOTFIX: Fix header in ByteArrayConverter
    
    Author: Ismael Juma 
    
    Reviewers: Manikumar reddy O 
    
    Closes #2690 from ijuma/fix-header-in-byte-array-converter
    ---
     .../kafka/connect/converters/ByteArrayConverter.java | 12 ++++++------
     .../connect/converters/ByteArrayConverterTest.java   | 10 +++++-----
     2 files changed, 11 insertions(+), 11 deletions(-)
    
    diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java
    index f592624cf2ffb..05dff27dce220 100644
    --- a/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java
    +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java
    @@ -1,19 +1,19 @@
    -/**
    +/*
      * 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,
      * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
      * See the License for the specific language governing permissions and
      * limitations under the License.
    - **/
    + */
     
     package org.apache.kafka.connect.converters;
     
    @@ -49,4 +49,4 @@ public SchemaAndValue toConnectData(String topic, byte[] value) {
             return new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value);
         }
     
    -}
    \ No newline at end of file
    +}
    diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java
    index e0d533f762225..b0a603a5b8c40 100644
    --- a/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java
    +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java
    @@ -1,19 +1,19 @@
    -/**
    +/*
      * 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,
      * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
      * See the License for the specific language governing permissions and
      * limitations under the License.
    - **/
    + */
     
     package org.apache.kafka.connect.converters;
     
    
    From b9f812491f5eb06ffe5b68f5e53df4302d2f68a8 Mon Sep 17 00:00:00 2001
    From: Guozhang Wang 
    Date: Wed, 15 Mar 2017 10:46:57 -0700
    Subject: [PATCH 097/101] MINOR: Improve log4j on stream thread and stream
     process
    
    Author: Guozhang Wang 
    
    Reviewers: Matthias J. Sax , Eno Thereska , Damian Guy , Jason Gustafson 
    
    Closes #2685 from guozhangwang/KMinor-improve-log4j
    ---
     .../apache/kafka/streams/KafkaStreams.java    | 24 ++++++++----
     .../processor/internals/StreamThread.java     | 39 ++++++++-----------
     2 files changed, 32 insertions(+), 31 deletions(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    index 8d8626da088c5..b23d2440fc500 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
    @@ -131,6 +131,7 @@ public class KafkaStreams {
         // of the co-location of stream thread's consumers. It is for internal
         // usage only and should not be exposed to users at all.
         private final UUID processId;
    +    private final String logPrefix;
         private final StreamsMetadataState streamsMetadataState;
     
         private final StreamsConfig config;
    @@ -217,9 +218,13 @@ public void setStateListener(final StateListener listener) {
         private synchronized void setState(final State newState) {
             final State oldState = state;
             if (!state.isValidTransition(newState)) {
    -            log.warn("Unexpected state transition from {} to {}.", oldState, newState);
    +            log.warn("{} Unexpected state transition from {} to {}.", logPrefix, oldState, newState);
    +        } else {
    +            log.info("{} State transition from {} to {}.", logPrefix, oldState, newState);
             }
    +
             state = newState;
    +
             if (stateListener != null) {
                 stateListener.onChange(state, oldState);
             }
    @@ -310,6 +315,8 @@ public KafkaStreams(final TopologyBuilder builder,
             if (clientId.length() <= 0)
                 clientId = applicationId + "-" + processId;
     
    +        this.logPrefix = String.format("stream-client [%s]", clientId);
    +
             final List reporters = config.getConfiguredInstances(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG,
                 MetricsReporter.class);
             reporters.add(new JmxReporter(JMX_PREFIX));
    @@ -329,7 +336,7 @@ public KafkaStreams(final TopologyBuilder builder,
             final ProcessorTopology globalTaskTopology = builder.buildGlobalStateTopology();
     
             if (config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) < 0) {
    -            log.warn("Negative cache size passed in. Reverting to cache size of 0 bytes.");
    +            log.warn("{} Negative cache size passed in. Reverting to cache size of 0 bytes.", logPrefix);
             }
     
             final long cacheSizeBytes = Math.max(0, config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) /
    @@ -395,7 +402,7 @@ private void checkBrokerVersionCompatibility() throws StreamsException {
             try {
                 client.close();
             } catch (final IOException e) {
    -            log.warn("Could not close StreamKafkaClient.", e);
    +            log.warn("{} Could not close StreamKafkaClient.", logPrefix, e);
             }
     
         }
    @@ -411,7 +418,7 @@ private void checkBrokerVersionCompatibility() throws StreamsException {
          * @throws StreamsException if the Kafka brokers have version 0.10.0.x
          */
         public synchronized void start() throws IllegalStateException, StreamsException {
    -        log.debug("Starting Kafka Stream process.");
    +        log.debug("{} Starting Kafka Stream process.", logPrefix);
     
             if (state == State.CREATED) {
                 checkBrokerVersionCompatibility();
    @@ -425,7 +432,7 @@ public synchronized void start() throws IllegalStateException, StreamsException
                     thread.start();
                 }
     
    -            log.info("Started Kafka Stream process");
    +            log.info("{} Started Kafka Stream process", logPrefix);
             } else {
                 throw new IllegalStateException("Cannot start again.");
             }
    @@ -450,7 +457,7 @@ public void close() {
          * before all threads stopped
          */
         public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
    -        log.debug("Stopping Kafka Stream process.");
    +        log.debug("{} Stopping Kafka Stream process.", logPrefix);
             if (state.isCreatedOrRunning()) {
                 setState(State.PENDING_SHUTDOWN);
                 // save the current thread so that if it is a stream thread
    @@ -486,7 +493,7 @@ public void run() {
                         }
     
                         metrics.close();
    -                    log.info("Stopped Kafka Streams process.");
    +                    log.info("{} Stopped Kafka Streams process.", logPrefix);
                     }
                 }, "kafka-streams-close-thread");
                 shutdown.setDaemon(true);
    @@ -556,7 +563,8 @@ public void cleanUp() {
             final String stateDir = config.getString(StreamsConfig.STATE_DIR_CONFIG);
     
             final String localApplicationDir = stateDir + File.separator + appId;
    -        log.debug("Removing local Kafka Streams application data in {} for application {}.",
    +        log.debug("{} Removing local Kafka Streams application data in {} for application {}.",
    +            logPrefix,
                 localApplicationDir,
                 appId);
     
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
    index 9a2c3fac1b116..6a6b508166e3e 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
    @@ -156,8 +156,11 @@ public synchronized State state() {
         private synchronized void setState(State newState) {
             State oldState = state;
             if (!state.isValidTransition(newState)) {
    -            log.warn("Unexpected state transition from " + state + " to " + newState);
    +            log.warn("Unexpected state transition from {} to {}.", logPrefix, oldState, newState);
    +        } else {
    +            log.info("{} State transition from {} to {}.", logPrefix, oldState, newState);
             }
    +
             state = newState;
             if (stateListener != null) {
                 stateListener.onChange(this, state, oldState);
    @@ -296,7 +299,7 @@ public StreamThread(TopologyBuilder builder,
             this.lastCleanMs = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment
             this.lastCommitMs = timerStartedMs;
             this.rebalanceListener = new RebalanceListener(time, config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG));
    -        setState(state.RUNNING);
    +        setState(State.RUNNING);
     
         }
     
    @@ -366,7 +369,7 @@ private void shutdown() {
             try {
                 partitionAssignor.close();
             } catch (Throwable e) {
    -            log.error("stream-thread [{}] Failed to close KafkaStreamClient: ", this.getName(), e);
    +            log.error("{} Failed to close KafkaStreamClient: ", logPrefix, e);
             }
     
             removeStreamTasks();
    @@ -393,7 +396,7 @@ private RuntimeException unAssignChangeLogPartitions() {
     
         @SuppressWarnings("ThrowableNotThrown")
         private void shutdownTasksAndState() {
    -        log.debug("{} shutdownTasksAndState: shutting down all active tasks [{}] and standby tasks [{}]", logPrefix,
    +        log.debug("{} shutdownTasksAndState: shutting down all active tasks {} and standby tasks {}", logPrefix,
                 activeTasks.keySet(), standbyTasks.keySet());
     
             final AtomicReference firstException = new AtomicReference<>(null);
    @@ -418,7 +421,7 @@ private void shutdownTasksAndState() {
          * soon the tasks will be assigned again
          */
         private void suspendTasksAndState()  {
    -        log.debug("{} suspendTasksAndState: suspending all active tasks [{}] and standby tasks [{}]", logPrefix,
    +        log.debug("{} suspendTasksAndState: suspending all active tasks {} and standby tasks {}", logPrefix,
                 activeTasks.keySet(), standbyTasks.keySet());
             final AtomicReference firstException = new AtomicReference<>(null);
             // Close all topology nodes
    @@ -709,7 +712,7 @@ protected void maybeClean(final long now) {
          * Commit the states of all its tasks
          */
         private void commitAll() {
    -        log.trace("stream-thread [{}] Committing all its owned tasks", this.getName());
    +        log.trace("{} Committing all its owned tasks", logPrefix);
             for (StreamTask task : activeTasks.values()) {
                 commitOne(task);
             }
    @@ -775,7 +778,7 @@ public Set cachedTasks() {
         }
     
         protected StreamTask createStreamTask(TaskId id, Collection partitions) {
    -        log.info("{} Creating active task {} with assigned partitions [{}]", logPrefix, id, partitions);
    +        log.info("{} Creating active task {} with assigned partitions {}", logPrefix, id, partitions);
     
             streamsMetrics.taskCreatedSensor.record();
     
    @@ -894,7 +897,7 @@ private void addStreamTasks(Collection assignment) {
         }
     
         StandbyTask createStandbyTask(TaskId id, Collection partitions) {
    -        log.info("{} Creating new standby task {} with assigned partitions [{}]", logPrefix, id, partitions);
    +        log.info("{} Creating new standby task {} with assigned partitions {}", logPrefix, id, partitions);
     
             streamsMetrics.taskCreatedSensor.record();
     
    @@ -965,14 +968,14 @@ private void updateStandByTaskMaps(final Map checkpointedO
         }
     
         private void updateSuspendedTasks() {
    -        log.info("{} Updating suspended tasks to contain active tasks [{}]", logPrefix, activeTasks.keySet());
    +        log.info("{} Updating suspended tasks to contain active tasks {}", logPrefix, activeTasks.keySet());
             suspendedTasks.clear();
             suspendedTasks.putAll(activeTasks);
             suspendedStandbyTasks.putAll(standbyTasks);
         }
     
         private void removeStreamTasks() {
    -        log.info("{} Removing all active tasks [{}]", logPrefix, activeTasks.keySet());
    +        log.info("{} Removing all active tasks {}", logPrefix, activeTasks.keySet());
     
             try {
                 prevActiveTasks.clear();
    @@ -987,7 +990,7 @@ private void removeStreamTasks() {
         }
     
         private void removeStandbyTasks() {
    -        log.info("{} Removing all standby tasks [{}]", logPrefix, standbyTasks.keySet());
    +        log.info("{} Removing all standby tasks {}", logPrefix, standbyTasks.keySet());
     
             standbyTasks.clear();
             standbyTasksByPartition.clear();
    @@ -1197,12 +1200,7 @@ private class RebalanceListener implements ConsumerRebalanceListener {
             public void onPartitionsAssigned(Collection assignment) {
                 final long start = time.milliseconds();
                 try {
    -                if (state == State.PENDING_SHUTDOWN) {
    -                    log.info("stream-thread [{}] New partitions [{}] assigned while shutting down.",
    -                        StreamThread.this.getName(), assignment);
    -                }
    -                log.info("stream-thread [{}] New partitions [{}] assigned at the end of consumer rebalance.",
    -                    StreamThread.this.getName(), assignment);
    +                log.info("{} at state {}: new partitions {} assigned at the end of consumer rebalance.", logPrefix, state, assignment);
                     storeChangelogReader = new StoreChangelogReader(restoreConsumer, time, requestTimeOut);
                     setStateWhenNotInPendingShutdown(State.ASSIGNING_PARTITIONS);
                     // do this first as we may have suspended standby tasks that
    @@ -1226,12 +1224,7 @@ public void onPartitionsAssigned(Collection assignment) {
             @Override
             public void onPartitionsRevoked(Collection assignment) {
                 try {
    -                if (state == State.PENDING_SHUTDOWN) {
    -                    log.info("stream-thread [{}] New partitions [{}] revoked while shutting down.",
    -                             StreamThread.this.getName(), assignment);
    -                }
    -                log.info("stream-thread [{}] partitions [{}] revoked at the beginning of consumer rebalance.",
    -                         StreamThread.this.getName(), assignment);
    +                log.info("{} at state {}: partitions {} revoked at the beginning of consumer rebalance.", logPrefix, state, assignment);
                     setStateWhenNotInPendingShutdown(State.PARTITIONS_REVOKED);
                     lastCleanMs = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned
                     // suspend active tasks
    
    From be11272818a06eea157ab7f6eee8905855f1cede Mon Sep 17 00:00:00 2001
    From: Vahid Hashemian 
    Date: Wed, 15 Mar 2017 11:50:57 -0700
    Subject: [PATCH 098/101] MINOR: Fix a documentation typo
    
    Author: Vahid Hashemian 
    
    Reviewers: Sriharsha Chintalapani 
    
    Closes #2674 from vahidhashemian/minor/fix_typos_1703
    ---
     docs/ops.html | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/docs/ops.html b/docs/ops.html
    index 9232f651ecae1..7be9939b154c8 100644
    --- a/docs/ops.html
    +++ b/docs/ops.html
    @@ -33,7 +33,7 @@ 

    Adding and removing The replication factor controls how many servers will replicate each message that is written. If you have a replication factor of 3 then up to 2 servers can fail before you will lose access to your data. We recommend you use a replication factor of 2 or 3 so that you can transparently bounce machines without interrupting data consumption.

    - The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (noi counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the concepts section. + The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (not counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the concepts section.

    Each sharded partition log is placed into its own folder under the Kafka log directory. The name of such folders consists of the topic name, appended by a dash (-) and the partition id. Since a typical folder name can not be over 255 characters long, there will be a limitation on the length of topic names. We assume the number of partitions will not ever be above 100,000. Therefore, topic names cannot be longer than 249 characters. This leaves just enough room in the folder name for a dash and a potentially 5 digit long partition id.

    From 7c38fc0638059c3844b2eaf56675be8d1cb149ea Mon Sep 17 00:00:00 2001 From: garmes-gdev Date: Thu, 16 Mar 2017 09:32:23 +0100 Subject: [PATCH 099/101] fix and tests (status: PASS) --- core/src/main/scala/kafka/tools/GetOffsetShell.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index ef90e4b8343c2..84b01b49c2b55 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -119,7 +119,7 @@ object GetOffsetShell { val partitions:java.util.Map[TopicPartition, java.lang.Long] = Map(new TopicPartition(metadata.topic(), metadata.partition()) -> java.lang.Long.valueOf(time)).asJava - val request: ListOffsetRequest.Builder = new ListOffsetRequest.Builder().setMinVersion(0.toShort).setTargetTimes(partitions) + val request= new ListOffsetRequest.Builder().setMinVersion(0.toShort).setTargetTimes(partitions) val listOffset= adminClient.getTopicListOffset(request,metadata.leader() ) From c659e8e754dfadec3fc8c1bf9966c998e25e8335 Mon Sep 17 00:00:00 2001 From: garmes-gdev Date: Thu, 16 Mar 2017 12:10:40 +0100 Subject: [PATCH 100/101] rebase --- .../main/scala/kafka/admin/AdminClient.scala | 11 ++ .../scala/kafka/tools/GetOffsetShell.scala | 164 +++++++++++------- tests/kafkatest/services/kafka/kafka.py | 4 +- .../tests/core/get_offset_shell_test.py | 4 +- 4 files changed, 114 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 4b284608a258a..4c3edd77b9282 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -220,6 +220,17 @@ class AdminClient(val time: Time, ConsumerGroupSummary(metadata.state, metadata.protocol, Some(consumers), coordinator) } + def getTopicListOffset(req: ListOffsetRequest.Builder, node: Node): Map[TopicPartition, ListOffsetResponse.PartitionData] = { + val responseBody = send(node, ApiKeys.LIST_OFFSETS, req) + responseBody.asInstanceOf[ListOffsetResponse].responseData().asScala.toMap + } + + + def getMetadata(req: MetadataRequest.Builder, node: Node): MetadataResponse = { + val responseBody = send(node, ApiKeys.METADATA, req) + responseBody.asInstanceOf[MetadataResponse] + } + def close() { client.close() } diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 9a19b1fa9195d..541e5f6fccbe6 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -18,92 +18,126 @@ */ package kafka.tools -import kafka.consumer._ +import java.util.Properties import joptsimple._ -import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo} -import kafka.common.TopicAndPartition +import kafka.admin.AdminClient import kafka.client.ClientUtils -import kafka.utils.{CommandLineUtils, Exit, ToolsUtils} +import kafka.cluster.BrokerEndPoint +import kafka.utils.{CommandLineUtils, ToolsUtils} +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.requests.{ListOffsetRequest, MetadataRequest} +import org.apache.kafka.common.utils.Utils +import scala.collection.JavaConverters._ +import scala.util.Random + object GetOffsetShell { + val clientId = "GetOffsetShell" + + private def createAdminClient(props: Properties): AdminClient = { + AdminClient.create(props) + } + private def getNode(brokerEndPoint: BrokerEndPoint): Node = { + new Node(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + } + + def main(args: Array[String]): Unit = { val parser = new OptionParser val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.") - .withRequiredArg - .describedAs("hostname:port,...,hostname:port") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("hostname:port,...,hostname:port") + .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to get offset from.") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) val partitionOpt = parser.accepts("partitions", "comma separated list of partition ids. If not specified, it will find offsets for all partitions") - .withRequiredArg - .describedAs("partition ids") - .ofType(classOf[String]) - .defaultsTo("") - val timeOpt = parser.accepts("time", "timestamp of the offsets before that") - .withRequiredArg - .describedAs("timestamp/-1(latest)/-2(earliest)") - .ofType(classOf[java.lang.Long]) - .defaultsTo(-1) - val nOffsetsOpt = parser.accepts("offsets", "number of offsets returned") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) - val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.") - .withRequiredArg - .describedAs("ms") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1000) - - if(args.length == 0) + .withRequiredArg + .describedAs("partition ids") + .ofType(classOf[String]) + .defaultsTo("") + val timeOpt = parser.accepts("time", " REQUIRED: timestamp of the offsets before that") + .withRequiredArg + .describedAs("timestamp/-1(latest)/-2(earliest)") + .ofType(classOf[java.lang.Long]) + + val commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client.") + .withRequiredArg + .describedAs("command config property file") + .ofType(classOf[String]) + + if(args.length == 0) CommandLineUtils.printUsageAndDie(parser, "An interactive shell for getting consumer offsets.") val options = parser.parse(args : _*) - CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt) + CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt, topicOpt, timeOpt) + - val clientId = "GetOffsetShell" val brokerList = options.valueOf(brokerListOpt) ToolsUtils.validatePortOrDie(parser, brokerList) val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) val topic = options.valueOf(topicOpt) - var partitionList = options.valueOf(partitionOpt) - var time = options.valueOf(timeOpt).longValue - val nOffsets = options.valueOf(nOffsetsOpt).intValue - val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() - - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata - if(topicsMetadata.size != 1 || !topicsMetadata.head.topic.equals(topic)) { - System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + - "kafka-list-topic.sh to verify") - Exit.exit(1) - } - val partitions = - if(partitionList == "") { - topicsMetadata.head.partitionsMetadata.map(_.partitionId) - } else { - partitionList.split(",").map(_.toInt).toSeq - } - partitions.foreach { partitionId => - val partitionMetadataOpt = topicsMetadata.head.partitionsMetadata.find(_.partitionId == partitionId) - partitionMetadataOpt match { - case Some(metadata) => - metadata.leader match { - case Some(leader) => - val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId) - val topicAndPartition = TopicAndPartition(topic, partitionId) - val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) - val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets - - println("%s:%d:%s".format(topic, partitionId, offsets.mkString(","))) - case None => System.err.println("Error: partition %d does not have a leader. Skip getting offsets".format(partitionId)) + + val partitionList = options.valueOf(partitionOpt) + val time = options.valueOf(timeOpt).longValue + val commandConfig = if (options.has(commandConfigOpt)) { + Utils.loadProps(options.valueOf(commandConfigOpt)) + } else new Properties() + + + commandConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList) + val adminClient = createAdminClient(commandConfig) + + val shuffledBrokers = Random.shuffle(metadataTargetBrokers) + + val metadataRes = adminClient.getMetadata(new MetadataRequest.Builder(List(topic).asJava), getNode(shuffledBrokers(0))) + + if(metadataRes.errors.containsKey(topic)){ + metadataRes.errors().get(topic).exception() + }else{ + + val topicsPartitions = metadataRes.cluster().availablePartitionsForTopic(topic).asScala + + val partitions = + if(partitionList == "") { + topicsPartitions.map(_.partition()) + } else { + partitionList.split(",").map(_.toInt).toSeq + } + + partitions.foreach { partitionId: Int => + val partitionMetadata = topicsPartitions.toList.find(_.partition == partitionId) + partitionMetadata match { + case Some(metadata) => { + + val partitions:java.util.Map[TopicPartition, java.lang.Long] = Map(new TopicPartition(metadata.topic(), metadata.partition()) -> + java.lang.Long.valueOf(time)).asJava + + val request= ListOffsetRequest.Builder.forConsumer(true).setTargetTimes(partitions) + + val listOffset= adminClient.getTopicListOffset(request,metadata.leader() ) + + listOffset.keys.foreach(topicPartition =>{ + val data = listOffset.get(topicPartition).get + + if (data.error.code() == Errors.NONE.code) { + println("%s:%d:%s".format(topic, partitionId, data.offset )) + } else { + val errormessage =Errors.forCode(data.error.code()).exception.getMessage + println(s"Attempt to fetch offsets for partition $topicPartition failed due to: $errormessage") + } + }) + } - case None => System.err.println("Error: partition %d does not exist".format(partitionId)) + } } } + } -} +} \ No newline at end of file diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 7e4a5899e6c48..12285966f7fc7 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -578,12 +578,12 @@ def is_registered(self, node): self.logger.debug("Broker info: %s", broker_info) return broker_info is not None - def get_offset_shell(self, topic, partitions, max_wait_ms, offsets, time): + def get_offset_shell(self, topic, partitions, time): node = self.nodes[0] cmd = self.path.script("kafka-run-class.sh", node) cmd += " kafka.tools.GetOffsetShell" - cmd += " --topic %s --broker-list %s --max-wait-ms %s --offsets %s --time %s" % (topic, self.bootstrap_servers(self.security_protocol), max_wait_ms, offsets, time) + cmd += " --topic %s --broker-list %s --time %s" % (topic, self.bootstrap_servers(self.security_protocol), time) if partitions: cmd += ' --partitions %s' % partitions diff --git a/tests/kafkatest/tests/core/get_offset_shell_test.py b/tests/kafkatest/tests/core/get_offset_shell_test.py index e45365d32cf31..d139f766e2c13 100644 --- a/tests/kafkatest/tests/core/get_offset_shell_test.py +++ b/tests/kafkatest/tests/core/get_offset_shell_test.py @@ -80,7 +80,7 @@ def test_get_offset_shell(self, security_protocol='PLAINTEXT'): self.start_producer() # Assert that offset fetched without any consumers consuming is 0 - assert self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, 0) + assert self.kafka.get_offset_shell(TOPIC, None, -1), "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, 0) self.start_consumer(security_protocol) @@ -89,5 +89,5 @@ def test_get_offset_shell(self, security_protocol='PLAINTEXT'): wait_until(lambda: self.consumer.alive(node), timeout_sec=10, backoff_sec=.2, err_msg="Consumer was too slow to start") # Assert that offset is correctly indicated by GetOffsetShell tool - wait_until(lambda: "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, MAX_MESSAGES) in self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), timeout_sec=10, + wait_until(lambda: "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, MAX_MESSAGES) in self.kafka.get_offset_shell(TOPIC, None, -1), timeout_sec=10, err_msg="Timed out waiting to reach expected offset.") \ No newline at end of file From 3e8bfcafbd5e07bbb441fb7aec7af69ae12f7204 Mon Sep 17 00:00:00 2001 From: garmes-gdev Date: Thu, 16 Mar 2017 14:17:01 +0100 Subject: [PATCH 101/101] fix --- core/src/main/scala/kafka/tools/GetOffsetShell.scala | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 8911ab43b1cec..bb9f617bdd848 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -140,11 +140,5 @@ object GetOffsetShell { } } -<<<<<<< HEAD -} -======= -} - - ->>>>>>> 7c38fc0638059c3844b2eaf56675be8d1cb149ea +}