From a670537aa33732b15b56644d8ccc1681e16395f5 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Fri, 14 Mar 2014 15:14:33 -0700 Subject: [PATCH] KAFKA-1012; Consumer offset management in Kafka; patched by Tejas Patil and Joel Koshy; feedback and reviews from Neha Narkhede, Jun Rao, Guozhang Wang, Sriram Subramanian, Joe Stein, Chris Riccomini --- .../main/scala/kafka/admin/TopicCommand.scala | 7 +- .../kafka/api/ConsumerMetadataRequest.scala | 79 +++ .../kafka/api/ConsumerMetadataResponse.scala | 57 +++ .../scala/kafka/api/OffsetCommitRequest.scala | 57 ++- .../kafka/api/OffsetCommitResponse.scala | 40 +- .../scala/kafka/api/OffsetFetchRequest.scala | 20 +- .../main/scala/kafka/api/RequestKeys.scala | 4 +- .../main/scala/kafka/client/ClientUtils.scala | 100 +++- .../main/scala/kafka/cluster/Partition.scala | 28 +- ...umerCoordinatorNotAvailableException.scala | 22 + .../scala/kafka/common/ErrorMapping.scala | 8 +- .../NotCoordinatorForConsumerException.scala | 22 + .../kafka/common/OffsetMetadataAndError.scala | 41 +- .../OffsetsLoadInProgressException.scala | 26 + core/src/main/scala/kafka/common/Topic.scala | 4 + .../kafka/consumer/ConsoleConsumer.scala | 16 +- .../scala/kafka/consumer/ConsumerConfig.scala | 39 ++ .../kafka/consumer/ConsumerConnector.scala | 2 +- .../scala/kafka/consumer/SimpleConsumer.scala | 11 +- .../scala/kafka/consumer/TopicCount.scala | 14 +- .../scala/kafka/consumer/TopicFilter.scala | 11 +- .../consumer/ZookeeperConsumerConnector.scala | 300 ++++++++--- .../kafka/controller/KafkaController.scala | 4 +- .../javaapi/ConsumerMetadataResponse.scala | 42 ++ .../kafka/javaapi/OffsetCommitRequest.scala | 13 +- .../kafka/javaapi/OffsetCommitResponse.scala | 2 +- .../javaapi/consumer/ConsumerConnector.java | 6 +- .../consumer/ZookeeperConsumerConnector.scala | 8 +- .../main/scala/kafka/log/FileMessageSet.scala | 4 +- .../scala/kafka/producer/ProducerConfig.scala | 11 +- .../main/scala/kafka/server/KafkaApis.scala | 324 ++++++------ .../main/scala/kafka/server/KafkaConfig.scala | 44 +- .../main/scala/kafka/server/KafkaServer.scala | 30 +- .../scala/kafka/server/OffsetManager.scala | 480 ++++++++++++++++++ .../scala/kafka/server/ReplicaManager.scala | 19 +- .../kafka/tools/ConsumerOffsetChecker.scala | 93 +++- .../scala/kafka/tools/DumpLogSegments.scala | 8 +- .../main/scala/kafka/tools/MirrorMaker.scala | 1 - .../kafka/tools/ReplicaVerificationTool.scala | 5 +- .../kafka/tools/VerifyConsumerRebalance.scala | 4 +- .../kafka/utils/VerifiableProperties.scala | 20 + core/src/main/scala/kafka/utils/ZkUtils.scala | 4 +- .../scala/other/kafka/TestOffsetManager.scala | 291 +++++++++++ .../other/kafka/TestZKConsumerOffsets.scala | 73 --- .../unit/kafka/admin/DeleteTopicTest.scala | 7 - .../RequestResponseSerializationTest.scala | 155 ++---- .../unit/kafka/consumer/TopicFilterTest.scala | 24 +- .../ZookeeperConsumerConnectorTest.scala | 4 +- .../unit/kafka/server/OffsetCommitTest.scala | 111 ++-- .../unit/kafka/server/SimpleFetchTest.scala | 9 +- .../migration_tool_test.py | 2 +- system_test/mirror_maker/README | 22 - system_test/mirror_maker/bin/expected.out | 18 - system_test/mirror_maker/bin/run-test.sh | 357 ------------- .../config/blacklisttest.consumer.properties | 28 - .../config/mirror_producer.properties | 30 -- .../config/server_source_1_1.properties | 76 --- .../config/server_source_1_2.properties | 76 --- .../config/server_source_2_1.properties | 76 --- .../config/server_source_2_2.properties | 76 --- .../config/server_target_1_1.properties | 78 --- .../config/server_target_1_2.properties | 78 --- .../whitelisttest_1.consumer.properties | 28 - .../whitelisttest_2.consumer.properties | 28 - .../config/zookeeper_target.properties | 18 - .../mirror_maker_test.py | 2 +- .../cluster_config.json | 103 ++++ .../config/console_consumer.properties | 2 + .../config/producer_performance.properties | 0 .../config/server.properties | 144 ++++++ .../config/zookeeper.properties} | 7 +- .../offset_management_test.py | 298 +++++++++++ .../testcase_7001_properties.json | 95 ++++ .../config/kafka_server_1.properties | 148 ++++++ .../config/kafka_server_2.properties | 148 ++++++ .../config/kafka_server_3.properties | 148 ++++++ .../config/kafka_server_4.properties | 148 ++++++ .../config/zookeeper_0.properties} | 10 +- .../testcase_7002_properties.json | 127 +++++ .../replica_basic_test.py | 2 +- system_test/utils/kafka_system_test_utils.py | 170 ++++++- system_test/utils/testcase_env.py | 6 + 82 files changed, 3609 insertions(+), 1644 deletions(-) create mode 100644 core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala create mode 100644 core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala create mode 100644 core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala create mode 100644 core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala create mode 100644 core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala create mode 100644 core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala create mode 100644 core/src/main/scala/kafka/server/OffsetManager.scala create mode 100644 core/src/test/scala/other/kafka/TestOffsetManager.scala delete mode 100644 core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala delete mode 100644 system_test/mirror_maker/README delete mode 100644 system_test/mirror_maker/bin/expected.out delete mode 100644 system_test/mirror_maker/bin/run-test.sh delete mode 100644 system_test/mirror_maker/config/blacklisttest.consumer.properties delete mode 100644 system_test/mirror_maker/config/mirror_producer.properties delete mode 100644 system_test/mirror_maker/config/server_source_1_1.properties delete mode 100644 system_test/mirror_maker/config/server_source_1_2.properties delete mode 100644 system_test/mirror_maker/config/server_source_2_1.properties delete mode 100644 system_test/mirror_maker/config/server_source_2_2.properties delete mode 100644 system_test/mirror_maker/config/server_target_1_1.properties delete mode 100644 system_test/mirror_maker/config/server_target_1_2.properties delete mode 100644 system_test/mirror_maker/config/whitelisttest_1.consumer.properties delete mode 100644 system_test/mirror_maker/config/whitelisttest_2.consumer.properties delete mode 100644 system_test/mirror_maker/config/zookeeper_target.properties create mode 100644 system_test/offset_management_testsuite/cluster_config.json create mode 100644 system_test/offset_management_testsuite/config/console_consumer.properties create mode 100644 system_test/offset_management_testsuite/config/producer_performance.properties create mode 100644 system_test/offset_management_testsuite/config/server.properties rename system_test/{mirror_maker/config/zookeeper_source_1.properties => offset_management_testsuite/config/zookeeper.properties} (83%) create mode 100644 system_test/offset_management_testsuite/offset_management_test.py create mode 100644 system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json create mode 100644 system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties create mode 100644 system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties create mode 100644 system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties create mode 100644 system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties rename system_test/{mirror_maker/config/zookeeper_source_2.properties => offset_management_testsuite/testcase_7002/config/zookeeper_0.properties} (79%) create mode 100644 system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index dc9b092a58c6..6fef9df19fbc 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -26,6 +26,8 @@ import scala.collection.JavaConversions._ import kafka.cluster.Broker import kafka.log.LogConfig import kafka.consumer.Whitelist +import kafka.server.OffsetManager + object TopicCommand { @@ -70,7 +72,7 @@ object TopicCommand { if (opts.options.has(opts.topicOpt)) { val topicsSpec = opts.options.valueOf(opts.topicOpt) val topicsFilter = new Whitelist(topicsSpec) - allTopics.filter(topicsFilter.isTopicAllowed) + allTopics.filter(topicsFilter.isTopicAllowed(_, excludeInternalTopics = false)) } else allTopics } @@ -104,6 +106,9 @@ object TopicCommand { println("Updated config for topic \"%s\".".format(topic)) } if(opts.options.has(opts.partitionsOpt)) { + if (topic == OffsetManager.OffsetsTopicName) { + throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.") + } println("WARNING: If partitions are increased for a topic that has a key, the partition " + "logic or ordering of the messages will be affected") val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala new file mode 100644 index 000000000000..dfad6e6534dd --- /dev/null +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.network.{BoundedByteBufferSend, RequestChannel} +import kafka.network.RequestChannel.Response +import kafka.common.ErrorMapping + +object ConsumerMetadataRequest { + val CurrentVersion = 0.shortValue + val DefaultClientId = "" + + def readFrom(buffer: ByteBuffer) = { + // envelope + val versionId = buffer.getShort + val correlationId = buffer.getInt + val clientId = ApiUtils.readShortString(buffer) + + // request + val group = ApiUtils.readShortString(buffer) + ConsumerMetadataRequest(group, versionId, correlationId, clientId) + } + +} + +case class ConsumerMetadataRequest(group: String, + versionId: Short = ConsumerMetadataRequest.CurrentVersion, + override val correlationId: Int = 0, + clientId: String = ConsumerMetadataRequest.DefaultClientId) + extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey), correlationId) { + + def sizeInBytes = + 2 + /* versionId */ + 4 + /* correlationId */ + ApiUtils.shortStringLength(clientId) + + ApiUtils.shortStringLength(group) + + def writeTo(buffer: ByteBuffer) { + // envelope + buffer.putShort(versionId) + buffer.putInt(correlationId) + ApiUtils.writeShortString(buffer, clientId) + + // consumer metadata request + ApiUtils.writeShortString(buffer, group) + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + // return ConsumerCoordinatorNotAvailable for all uncaught errors + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + def describe(details: Boolean) = { + val consumerMetadataRequest = new StringBuilder + consumerMetadataRequest.append("Name: " + this.getClass.getSimpleName) + consumerMetadataRequest.append("; Version: " + versionId) + consumerMetadataRequest.append("; CorrelationId: " + correlationId) + consumerMetadataRequest.append("; ClientId: " + clientId) + consumerMetadataRequest.append("; Group: " + group) + consumerMetadataRequest.toString() + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala new file mode 100644 index 000000000000..6807f9806fb1 --- /dev/null +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -0,0 +1,57 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.cluster.Broker +import kafka.common.ErrorMapping + +object ConsumerMetadataResponse { + val CurrentVersion = 0 + + def readFrom(buffer: ByteBuffer) = { + val correlationId = buffer.getInt + val errorCode = buffer.getShort + val coordinatorOpt = if (errorCode == ErrorMapping.NoError) + Some(Broker.readFrom(buffer)) + else + None + + ConsumerMetadataResponse(coordinatorOpt, errorCode, correlationId) + } + +} + +case class ConsumerMetadataResponse (coordinator: Option[Broker], errorCode: Short, override val correlationId: Int = 0) + extends RequestOrResponse(correlationId = correlationId) { + + def sizeInBytes = + 4 + /* correlationId */ + 2 + /* error code */ + coordinator.map(_.sizeInBytes).getOrElse(0) + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(correlationId) + buffer.putShort(errorCode) + if (errorCode == ErrorMapping.NoError) { + coordinator.get.writeTo(buffer) + } + } + + def describe(details: Boolean) = toString +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 4d1fa5cbfde9..9f6956e97b5e 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -18,17 +18,20 @@ package kafka.api import java.nio.ByteBuffer - import kafka.api.ApiUtils._ -import kafka.utils.Logging +import kafka.utils.{SystemTime, Logging} import kafka.network.{RequestChannel, BoundedByteBufferSend} -import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError} +import kafka.common.{OffsetAndMetadata, ErrorMapping, TopicAndPartition} import kafka.network.RequestChannel.Response +import scala.collection._ + object OffsetCommitRequest extends Logging { val CurrentVersion: Short = 0 val DefaultClientId = "" def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { + val now = SystemTime.milliseconds + // Read values from the envelope val versionId = buffer.getShort val correlationId = buffer.getInt @@ -43,23 +46,45 @@ object OffsetCommitRequest extends Logging { (1 to partitionCount).map(_ => { val partitionId = buffer.getInt val offset = buffer.getLong + val timestamp = { + val given = buffer.getLong + if (given == -1L) now else given + } val metadata = readShortString(buffer) - (TopicAndPartition(topic, partitionId), OffsetMetadataAndError(offset, metadata)) + (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp)) }) }) - OffsetCommitRequest(consumerGroupId, Map(pairs:_*), versionId, correlationId, clientId) + OffsetCommitRequest(consumerGroupId, mutable.Map(pairs:_*), versionId, correlationId, clientId) } } case class OffsetCommitRequest(groupId: String, - requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], + requestInfo: mutable.Map[TopicAndPartition, OffsetAndMetadata], versionId: Short = OffsetCommitRequest.CurrentVersion, override val correlationId: Int = 0, clientId: String = OffsetCommitRequest.DefaultClientId) extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) - + + def filterLargeMetadata(maxMetadataSize: Int) = + requestInfo.filter(info => info._2.metadata == null || info._2.metadata.length <= maxMetadataSize) + + def responseFor(errorCode: Short, offsetMetadataMaxSize: Int) = { + val commitStatus = requestInfo.map {info => + (info._1, if (info._2.metadata != null && info._2.metadata.length > offsetMetadataMaxSize) + ErrorMapping.OffsetMetadataTooLargeCode + else if (errorCode == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (errorCode == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else + errorCode) + }.toMap + OffsetCommitResponse(commitStatus, correlationId) + } + + def writeTo(buffer: ByteBuffer) { // Write envelope buffer.putShort(versionId) @@ -73,9 +98,10 @@ case class OffsetCommitRequest(groupId: String, writeShortString(buffer, t1._1) // topic buffer.putInt(t1._2.size) // number of partitions for this topic t1._2.foreach( t2 => { - buffer.putInt(t2._1.partition) // partition - buffer.putLong(t2._2.offset) // offset - writeShortString(buffer, t2._2.metadata) // metadata + buffer.putInt(t2._1.partition) + buffer.putLong(t2._2.offset) + buffer.putLong(t2._2.timestamp) + writeShortString(buffer, t2._2.metadata) }) }) } @@ -95,15 +121,14 @@ case class OffsetCommitRequest(groupId: String, innerCount + 4 /* partition */ + 8 /* offset */ + + 8 /* timestamp */ + shortStringLength(offsetAndMetadata._2.metadata) }) }) override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val responseMap = requestInfo.map { - case (topicAndPartition, offset) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - }.toMap - val errorResponse = OffsetCommitResponse(requestInfo=responseMap, correlationId=correlationId) + val errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + val errorResponse = responseFor(errorCode, Int.MaxValue) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) } @@ -119,7 +144,7 @@ case class OffsetCommitRequest(groupId: String, offsetCommitRequest.toString() } - override def toString(): String = { - describe(true) + override def toString = { + describe(details = true) } } diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala index 9e1795f9db15..4946e9729ecb 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala @@ -19,9 +19,8 @@ package kafka.api import java.nio.ByteBuffer -import kafka.api.ApiUtils._ -import kafka.common.TopicAndPartition import kafka.utils.Logging +import kafka.common.TopicAndPartition object OffsetCommitResponse extends Logging { val CurrentVersion: Short = 0 @@ -30,7 +29,7 @@ object OffsetCommitResponse extends Logging { val correlationId = buffer.getInt val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { - val topic = readShortString(buffer) + val topic = ApiUtils.readShortString(buffer) val partitionCount = buffer.getInt (1 to partitionCount).map(_ => { val partitionId = buffer.getInt @@ -42,37 +41,34 @@ object OffsetCommitResponse extends Logging { } } -case class OffsetCommitResponse(requestInfo: Map[TopicAndPartition, Short], +case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short], override val correlationId: Int = 0) extends RequestOrResponse(correlationId=correlationId) { - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) + lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic) def writeTo(buffer: ByteBuffer) { buffer.putInt(correlationId) - buffer.putInt(requestInfoGroupedByTopic.size) // number of topics - requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, Short] - writeShortString(buffer, t1._1) // topic - buffer.putInt(t1._2.size) // number of partitions for this topic - t1._2.foreach( t2 => { // TopicAndPartition -> Short - buffer.putInt(t2._1.partition) - buffer.putShort(t2._2) //error - }) - }) + buffer.putInt(commitStatusGroupedByTopic.size) + commitStatusGroupedByTopic.foreach { case(topic, statusMap) => + ApiUtils.writeShortString(buffer, topic) + buffer.putInt(statusMap.size) // partition count + statusMap.foreach { case(topicAndPartition, errorCode) => + buffer.putInt(topicAndPartition.partition) + buffer.putShort(errorCode) + } + } } override def sizeInBytes = 4 + /* correlationId */ 4 + /* topic count */ - requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { - val (topic, offsets) = topicAndOffsets + commitStatusGroupedByTopic.foldLeft(0)((count, partitionStatusMap) => { + val (topic, partitionStatus) = partitionStatusMap count + - shortStringLength(topic) + /* topic */ - 4 + /* number of partitions */ - offsets.size * ( - 4 + /* partition */ - 2 /* error */ - ) + ApiUtils.shortStringLength(topic) + + 4 + /* partition count */ + partitionStatus.size * ( 4 /* partition */ + 2 /* error code */) }) override def describe(details: Boolean):String = { toString } diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index 7036532db140..a32f8588ff02 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -23,7 +23,7 @@ import kafka.api.ApiUtils._ import kafka.utils.Logging import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response -import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition} +import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError, TopicAndPartition} object OffsetFetchRequest extends Logging { val CurrentVersion: Short = 0 val DefaultClientId = "" @@ -50,10 +50,10 @@ object OffsetFetchRequest extends Logging { } case class OffsetFetchRequest(groupId: String, - requestInfo: Seq[TopicAndPartition], - versionId: Short = OffsetFetchRequest.CurrentVersion, - override val correlationId: Int = 0, - clientId: String = OffsetFetchRequest.DefaultClientId) + requestInfo: Seq[TopicAndPartition], + versionId: Short = OffsetFetchRequest.CurrentVersion, + override val correlationId: Int = 0, + clientId: String = OffsetFetchRequest.DefaultClientId) extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey), correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic) @@ -91,8 +91,8 @@ case class OffsetFetchRequest(groupId: String, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val responseMap = requestInfo.map { case (topicAndPartition) => (topicAndPartition, OffsetMetadataAndError( - offset=OffsetMetadataAndError.InvalidOffset, - error=ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + offset = OffsetAndMetadata.InvalidOffset, + error = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) )) }.toMap val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId) @@ -111,7 +111,7 @@ case class OffsetFetchRequest(groupId: String, offsetFetchRequest.toString() } - override def toString(): String = { - describe(true) + override def toString: String = { + describe(details = true) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index c81214fec0bf..fbfc9d3aeaff 100644 --- a/core/src/main/scala/kafka/api/RequestKeys.scala +++ b/core/src/main/scala/kafka/api/RequestKeys.scala @@ -31,6 +31,7 @@ object RequestKeys { val ControlledShutdownKey: Short = 7 val OffsetCommitKey: Short = 8 val OffsetFetchKey: Short = 9 + val ConsumerMetadataKey: Short = 10 val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]= Map(ProduceKey -> ("Produce", ProducerRequest.readFrom), @@ -42,7 +43,8 @@ object RequestKeys { UpdateMetadataKey -> ("UpdateMetadata", UpdateMetadataRequest.readFrom), ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom), OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom), - OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom)) + OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom), + ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom)) def nameForKey(key: Short): String = { keyToNameAndDeserializerMap.get(key) match { diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 1d2f81be4f98..fc9e08423a41 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -20,12 +20,16 @@ import scala.collection._ import kafka.cluster._ import kafka.api._ import kafka.producer._ -import kafka.common.KafkaException +import kafka.common.{ErrorMapping, KafkaException} import kafka.utils.{Utils, Logging} import java.util.Properties import util.Random + import kafka.network.BlockingChannel + import kafka.utils.ZkUtils._ + import org.I0Itec.zkclient.ZkClient + import java.io.IOException -/** + /** * Helper functions common to clients (producer, consumer, or admin) */ object ClientUtils extends Logging{ @@ -103,5 +107,93 @@ object ClientUtils extends Logging{ new Broker(brokerId, hostName, port) }) } - -} \ No newline at end of file + + /** + * Creates a blocking channel to a random broker + */ + def channelToAnyBroker(zkClient: ZkClient, socketTimeoutMs: Int = 3000) : BlockingChannel = { + var channel: BlockingChannel = null + var connected = false + while (!connected) { + val allBrokers = getAllBrokersInCluster(zkClient) + Random.shuffle(allBrokers).find { broker => + trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) + try { + channel = new BlockingChannel(broker.host, broker.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs) + channel.connect() + debug("Created channel to broker %s:%d.".format(channel.host, channel.port)) + true + } catch { + case e: Exception => + if (channel != null) channel.disconnect() + channel = null + info("Error while creating channel to %s:%d.".format(broker.host, broker.port)) + false + } + } + connected = if (channel == null) false else true + } + + channel + } + + /** + * Creates a blocking channel to the offset manager of the given group + */ + def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = { + var queryChannel = channelToAnyBroker(zkClient) + + var offsetManagerChannelOpt: Option[BlockingChannel] = None + + while (!offsetManagerChannelOpt.isDefined) { + + var coordinatorOpt: Option[Broker] = None + + while (!coordinatorOpt.isDefined) { + try { + if (!queryChannel.isConnected) + queryChannel = channelToAnyBroker(zkClient) + debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) + queryChannel.send(ConsumerMetadataRequest(group)) + val response = queryChannel.receive() + val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.buffer) + debug("Consumer metadata response: " + consumerMetadataResponse.toString) + if (consumerMetadataResponse.errorCode == ErrorMapping.NoError) + coordinatorOpt = consumerMetadataResponse.coordinator + } + catch { + case ioe: IOException => + info("Failed to fetch consumer metadata from %s:%d.".format(queryChannel.host, queryChannel.port)) + queryChannel.disconnect() + } + } + + val coordinator = coordinatorOpt.get + if (coordinator.host == queryChannel.host && coordinator.port == queryChannel.port) { + offsetManagerChannelOpt = Some(queryChannel) + } else { + val connectString = "%s:%d".format(coordinator.host, coordinator.port) + var offsetManagerChannel: BlockingChannel = null + try { + debug("Connecting to offset manager %s.".format(connectString)) + offsetManagerChannel = new BlockingChannel(coordinator.host, coordinator.port, + BlockingChannel.UseDefaultBufferSize, + BlockingChannel.UseDefaultBufferSize, + socketTimeoutMs) + offsetManagerChannel.connect() + offsetManagerChannelOpt = Some(offsetManagerChannel) + queryChannel.disconnect() + } + catch { + case ioe: IOException => // offsets manager may have moved + info("Error while connecting to %s.".format(connectString)) + if (offsetManagerChannel != null) offsetManagerChannel.disconnect() + Thread.sleep(retryBackOffMs) + offsetManagerChannelOpt = None // just in case someone decides to change shutdownChannel to not swallow exceptions + } + } + } + + offsetManagerChannelOpt.get + } + } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 882b6da8a8e4..0b88f14c4855 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -22,7 +22,7 @@ import kafka.utils._ import java.lang.Object import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.ReplicaManager +import kafka.server.{OffsetManager, ReplicaManager} import com.yammer.metrics.core.Gauge import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController @@ -165,7 +165,8 @@ class Partition(val topic: String, * and setting the new leader and ISR */ def makeLeader(controllerId: Int, - partitionStateInfo: PartitionStateInfo, correlationId: Int): Boolean = { + partitionStateInfo: PartitionStateInfo, correlationId: Int, + offsetManager: OffsetManager): Boolean = { leaderIsrUpdateLock synchronized { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -186,6 +187,8 @@ class Partition(val topic: String, leaderReplicaIdOpt = Some(localBrokerId) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(getReplica().get) + if (topic == OffsetManager.OffsetsTopicName) + offsetManager.loadOffsetsFromLog(partitionId) true } } @@ -196,7 +199,7 @@ class Partition(val topic: String, */ def makeFollower(controllerId: Int, partitionStateInfo: PartitionStateInfo, - correlationId: Int): Boolean = { + correlationId: Int, offsetManager: OffsetManager): Boolean = { leaderIsrUpdateLock synchronized { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -212,12 +215,21 @@ class Partition(val topic: String, inSyncReplicas = Set.empty[Replica] leaderEpoch = leaderAndIsr.leaderEpoch zkVersion = leaderAndIsr.zkVersion + + leaderReplicaIdOpt.foreach { leaderReplica => + if (topic == OffsetManager.OffsetsTopicName && + /* if we are making a leader->follower transition */ + leaderReplica == localBrokerId) + offsetManager.clearOffsetsInPartition(partitionId) + } - if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) - return false; - - leaderReplicaIdOpt = Some(newLeaderBrokerId) - true + if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { + false + } + else { + leaderReplicaIdOpt = Some(newLeaderBrokerId) + true + } } } diff --git a/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala b/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala new file mode 100644 index 000000000000..8e02d264e944 --- /dev/null +++ b/core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.common + +class ConsumerCoordinatorNotAvailableException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index b0b5dcedc4da..5559d26ba2b9 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -43,6 +43,9 @@ object ErrorMapping { val StaleControllerEpochCode: Short = 11 val OffsetMetadataTooLargeCode: Short = 12 val StaleLeaderEpochCode: Short = 13 + val OffsetsLoadInProgressCode: Short = 14 + val ConsumerCoordinatorNotAvailableCode: Short = 15 + val NotCoordinatorForConsumerCode: Short = 16 private val exceptionToCode = Map[Class[Throwable], Short]( @@ -57,7 +60,10 @@ object ErrorMapping { classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode, classOf[MessageSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSizeTooLargeCode, classOf[ControllerMovedException].asInstanceOf[Class[Throwable]] -> StaleControllerEpochCode, - classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode + classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode, + classOf[OffsetsLoadInProgressException].asInstanceOf[Class[Throwable]] -> OffsetsLoadInProgressCode, + classOf[ConsumerCoordinatorNotAvailableException].asInstanceOf[Class[Throwable]] -> ConsumerCoordinatorNotAvailableCode, + classOf[NotCoordinatorForConsumerException].asInstanceOf[Class[Throwable]] -> NotCoordinatorForConsumerCode ).withDefaultValue(UnknownCode) /* invert the mapping */ diff --git a/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala b/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala new file mode 100644 index 000000000000..1eb74be038ea --- /dev/null +++ b/core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.common + +class NotCoordinatorForConsumerException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 59608a34202b..1586243d20d6 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -1,5 +1,3 @@ -package kafka.common - /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -17,20 +15,41 @@ package kafka.common * limitations under the License. */ -/** - * Convenience case class since (topic, partition) pairs are ubiquitous. - */ -case class OffsetMetadataAndError(offset: Long, metadata: String = OffsetMetadataAndError.NoMetadata, error: Short = ErrorMapping.NoError) { +package kafka.common - def this(tuple: (Long, String, Short)) = this(tuple._1, tuple._2, tuple._3) +case class OffsetAndMetadata(offset: Long, + metadata: String = OffsetAndMetadata.NoMetadata, + timestamp: Long = -1L) { + override def toString = "OffsetAndMetadata[%d,%s%s]" + .format(offset, + if (metadata != null && metadata.length > 0) metadata else "NO_METADATA", + if (timestamp == -1) "" else "," + timestamp.toString) +} - def asTuple = (offset, metadata, error) +object OffsetAndMetadata { + val InvalidOffset: Long = -1L + val NoMetadata: String = "" + val InvalidTime: Long = -1L +} + +case class OffsetMetadataAndError(offset: Long, + metadata: String = OffsetAndMetadata.NoMetadata, + error: Short = ErrorMapping.NoError) { + + def this(offsetMetadata: OffsetAndMetadata, error: Short) = + this(offsetMetadata.offset, offsetMetadata.metadata, error) - override def toString = "OffsetAndMetadata[%d,%s,%d]".format(offset, metadata, error) + def this(error: Short) = + this(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, error) + def asTuple = (offset, metadata, error) + + override def toString = "OffsetMetadataAndError[%d,%s,%d]".format(offset, metadata, error) } object OffsetMetadataAndError { - val InvalidOffset: Long = -1L; - val NoMetadata: String = ""; + val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError) + val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode) + val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode) } + diff --git a/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala b/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala new file mode 100644 index 000000000000..1c8e96eefc7f --- /dev/null +++ b/core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.common + +/** + * Indicates that offsets are currently being loaded from disk into the cache so offset fetch requests cannot be satisfied. + */ +class OffsetsLoadInProgressException(message: String) extends RuntimeException(message) { + def this() = this(null) +} + diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index c1b9f65114c7..ad759786d1c2 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -18,12 +18,16 @@ package kafka.common import util.matching.Regex +import kafka.server.OffsetManager + object Topic { val legalChars = "[a-zA-Z0-9\\._\\-]" private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") + val InternalTopics = Set(OffsetManager.OffsetsTopicName) + def validate(topic: String) { if (topic.length <= 0) throw new InvalidTopicException("topic name is illegal, can't be empty") diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala index dc066c23e9c2..0f62819be056 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -123,7 +123,13 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("metrics dictory") .ofType(classOf[java.lang.String]) - + val includeInternalTopicsOpt = parser.accepts("include-internal-topics", "Allow consuming internal topics.") + val offsetsStorageOpt = parser.accepts("offsets-storage", "Specify offsets storage backend (kafka/zookeeper).") + .withRequiredArg + .describedAs("Offsets storage method.") + .ofType(classOf[String]) + .defaultsTo("zookeeper") + val dualCommitEnabledOpt = parser.accepts("dual-commit-enabled", "If offsets storage is kafka and this is set, then commit to zookeeper as well.") val options: OptionSet = tryParse(parser, args) CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt) @@ -153,6 +159,7 @@ object ConsoleConsumer extends Logging { KafkaMetricsReporter.startReporters(verifiableProps) } + val offsetsStorage = options.valueOf(offsetsStorageOpt) val props = new Properties() props.put("group.id", options.valueOf(groupIdOpt)) props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) @@ -166,6 +173,13 @@ object ConsoleConsumer extends Logging { props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) props.put("consumer.timeout.ms", options.valueOf(consumerTimeoutMsOpt).toString) props.put("refresh.leader.backoff.ms", options.valueOf(refreshMetadataBackoffMsOpt).toString) + props.put("offsets.storage", offsetsStorage) + if (options.has(includeInternalTopicsOpt)) + props.put("exclude.internal.topics", "false") + if (options.has(dualCommitEnabledOpt)) + props.put("dual.commit.enabled", "true") + else + props.put("dual.commit.enabled", "false") val config = new ConsumerConfig(props) val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index e6875d6aa830..1cf2f62ba02e 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -41,9 +41,14 @@ object ConsumerConfig extends Config { val MirrorTopicsWhitelist = "" val MirrorTopicsBlacklist = "" val MirrorConsumerNumThreads = 1 + val OffsetsChannelBackoffMs = 1000 + val OffsetsChannelSocketTimeoutMs = 10000 + val OffsetsCommitMaxRetries = 5 + val OffsetsStorage = "zookeeper" val MirrorTopicsWhitelistProp = "mirror.topics.whitelist" val MirrorTopicsBlacklistProp = "mirror.topics.blacklist" + val ExcludeInternalTopics = true val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads" val DefaultClientId = "" @@ -51,6 +56,7 @@ object ConsumerConfig extends Config { validateClientId(config.clientId) validateGroupId(config.groupId) validateAutoOffsetReset(config.autoOffsetReset) + validateOffsetsStorage(config.offsetsStorage) } def validateClientId(clientId: String) { @@ -69,6 +75,15 @@ object ConsumerConfig extends Config { "Valid values are " + OffsetRequest.SmallestTimeString + " and " + OffsetRequest.LargestTimeString) } } + + def validateOffsetsStorage(storage: String) { + storage match { + case "zookeeper" => + case "kafka" => + case _ => throw new InvalidConfigException("Wrong value " + storage + " of offsets.storage in consumer config; " + + "Valid values are 'zookeeper' and 'kafka'") + } + } } class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) { @@ -122,6 +137,27 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** backoff time to refresh the leader of a partition after it loses the current leader */ val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", RefreshMetadataBackoffMs) + /** backoff time to reconnect the offsets channel or to retry offset fetches/commits */ + val offsetsChannelBackoffMs = props.getInt("offsets.channel.backoff.ms", OffsetsChannelBackoffMs) + /** socket timeout to use when reading responses for Offset Fetch/Commit requests. This timeout will also be used for + * the ConsumerMetdata requests that are used to query for the offset coordinator. */ + val offsetsChannelSocketTimeoutMs = props.getInt("offsets.channel.socket.timeout.ms", OffsetsChannelSocketTimeoutMs) + + /** Retry the offset commit up to this many times on failure. This retry count only applies to offset commits during + * shut-down. It does not apply to commits from the auto-commit thread. It also does not apply to attempts to query + * for the offset coordinator before committing offsets. i.e., if a consumer metadata request fails for any reason, + * it is retried and that retry does not count toward this limit. */ + val offsetsCommitMaxRetries = props.getInt("offsets.commit.max.retries", OffsetsCommitMaxRetries) + + /** Specify whether offsets should be committed to "zookeeper" (default) or "kafka" */ + val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase + + /** If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This + * is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any + * given consumer group, it is safe to turn this off after all instances within that group have been migrated to + * the new jar that commits offsets to the broker (instead of directly to ZooKeeper). */ + val dualCommitEnabled = props.getBoolean("dual.commit.enabled", if (offsetsStorage == "kafka") true else false) + /* what to do if an offset is out of range. smallest : automatically reset the offset to the smallest offset largest : automatically reset the offset to the largest offset @@ -136,6 +172,9 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( */ val clientId = props.getString("client.id", groupId) + /** Whether messages from internal topics (such as offsets) should be exposed to the consumer. */ + val excludeInternalTopics = props.getBoolean("exclude.internal.topics", ExcludeInternalTopics) + validate(this) } diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index 13c3f771e9a9..07677c1c2676 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala @@ -70,7 +70,7 @@ trait ConsumerConnector { /** * Commit the offsets of all broker partitions connected by this connector. */ - def commitOffsets + def commitOffsets(retryOnFailure: Boolean = true) /** * Shut down the connector diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index fa7caa76d404..0e6463221038 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -92,6 +92,11 @@ class SimpleConsumer(val host: String, TopicMetadataResponse.readFrom(response.buffer) } + def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = { + val response = sendRequest(request) + ConsumerMetadataResponse.readFrom(response.buffer) + } + /** * Fetch a set of messages from a topic. * @@ -126,7 +131,11 @@ class SimpleConsumer(val host: String, * @param request a [[kafka.api.OffsetCommitRequest]] object. * @return a [[kafka.api.OffsetCommitResponse]] object. */ - def commitOffsets(request: OffsetCommitRequest) = OffsetCommitResponse.readFrom(sendRequest(request).buffer) + def commitOffsets(request: OffsetCommitRequest) = { + // TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before + // we can commit offsets. + OffsetCommitResponse.readFrom(sendRequest(request).buffer) + } /** * Fetch offsets for a topic diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala index e33263378489..c79311097c5b 100644 --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ b/core/src/main/scala/kafka/consumer/TopicCount.scala @@ -47,7 +47,7 @@ private[kafka] object TopicCount extends Logging { val blackListPattern = "black_list" val staticPattern = "static" - def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient) : TopicCount = { + def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient, excludeInternalTopics: Boolean) : TopicCount = { val dirs = new ZKGroupDirs(group) val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)._1 var subscriptionPattern: String = null @@ -85,15 +85,15 @@ private[kafka] object TopicCount extends Logging { new Whitelist(regex) else new Blacklist(regex) - new WildcardTopicCount(zkClient, consumerId, filter, numStreams) + new WildcardTopicCount(zkClient, consumerId, filter, numStreams, excludeInternalTopics) } } def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) = new StaticTopicCount(consumerIdString, topicCount) - def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient) = - new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams) + def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient, excludeInternalTopics: Boolean) = + new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams, excludeInternalTopics) } @@ -119,9 +119,11 @@ private[kafka] class StaticTopicCount(val consumerIdString: String, private[kafka] class WildcardTopicCount(zkClient: ZkClient, consumerIdString: String, topicFilter: TopicFilter, - numStreams: Int) extends TopicCount { + numStreams: Int, + excludeInternalTopics: Boolean) extends TopicCount { def getConsumerThreadIdsPerTopic = { - val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath).filter(topicFilter.isTopicAllowed(_)) + val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath) + .filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics)) makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) } diff --git a/core/src/main/scala/kafka/consumer/TopicFilter.scala b/core/src/main/scala/kafka/consumer/TopicFilter.scala index 4f2082360cfb..5a13540699be 100644 --- a/core/src/main/scala/kafka/consumer/TopicFilter.scala +++ b/core/src/main/scala/kafka/consumer/TopicFilter.scala @@ -20,6 +20,7 @@ package kafka.consumer import kafka.utils.Logging import java.util.regex.{PatternSyntaxException, Pattern} +import kafka.common.Topic sealed abstract class TopicFilter(rawRegex: String) extends Logging { @@ -41,12 +42,12 @@ sealed abstract class TopicFilter(rawRegex: String) extends Logging { override def toString = regex - def isTopicAllowed(topic: String): Boolean + def isTopicAllowed(topic: String, excludeInternalTopics: Boolean): Boolean } case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { - override def isTopicAllowed(topic: String) = { - val allowed = topic.matches(regex) + override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = { + val allowed = topic.matches(regex) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics) debug("%s %s".format( topic, if (allowed) "allowed" else "filtered")) @@ -58,8 +59,8 @@ case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { } case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) { - override def isTopicAllowed(topic: String) = { - val allowed = !topic.matches(regex) + override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = { + val allowed = (!topic.matches(regex)) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics) debug("%s %s".format( topic, if (allowed) "allowed" else "filtered")) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 703b2e22605c..9a3db904915f 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -34,7 +34,11 @@ import kafka.utils.Utils.inLock import kafka.common._ import com.yammer.metrics.core.Gauge import kafka.metrics._ +import kafka.network.BlockingChannel +import kafka.client.ClientUtils +import kafka.api._ import scala.Some +import kafka.common.TopicAndPartition /** @@ -85,7 +89,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var fetcher: Option[ConsumerFetcherManager] = None private var zkClient: ZkClient = null private var topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] - private var checkpointedOffsets = new Pool[TopicAndPartition, Long] + private val checkpointedOffsets = new Pool[TopicAndPartition, Long] private val topicThreadIdAndQueues = new Pool[(String,String), BlockingQueue[FetchedDataChunk]] private val scheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "kafka-consumer-scheduler-") private val messageStreamCreated = new AtomicBoolean(false) @@ -94,8 +98,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var topicPartitionChangeListener: ZKTopicPartitionChangeListener = null private var loadBalancerListener: ZKRebalancerListener = null + private var offsetsChannel: BlockingChannel = null + private val offsetsChannelLock = new Object + private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null + // useful for tracking migration of consumers to store offsets in kafka + private val kafkaCommitMeter = newMeter(config.clientId + "-KafkaCommitsPerSec", "commits", TimeUnit.SECONDS) + private val zkCommitMeter = newMeter(config.clientId + "-ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS) + val consumerIdString = { var consumerUuid : String = null config.consumerId match { @@ -113,6 +124,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, connectZk() createFetcher() + ensureOffsetManagerConnected() + if (config.autoCommitEnable) { scheduler.startup info("starting auto committer every " + config.autoCommitIntervalMs + " ms") @@ -156,12 +169,22 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) } + // Blocks until the offset manager is located and a channel is established to it. + private def ensureOffsetManagerConnected() { + if (config.offsetsStorage == "kafka") { + if (offsetsChannel == null || !offsetsChannel.isConnected) + offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) + + debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port)) + } + } + def shutdown() { - rebalanceLock synchronized { - val canShutdown = isShuttingDown.compareAndSet(false, true); - if (canShutdown) { - info("ZKConsumerConnector shutting down") + val canShutdown = isShuttingDown.compareAndSet(false, true) + if (canShutdown) { + info("ZKConsumerConnector shutting down") + rebalanceLock synchronized { if (wildcardTopicWatcher != null) wildcardTopicWatcher.shutdown() try { @@ -178,6 +201,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, zkClient.close() zkClient = null } + + if (offsetsChannel != null) offsetsChannel.disconnect() } catch { case e: Throwable => fatal("error during consumer connector shutdown", e) @@ -240,7 +265,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def autoCommit() { trace("auto committing") try { - commitOffsets() + commitOffsets(isAutoCommit = false) } catch { case t: Throwable => @@ -249,30 +274,184 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - def commitOffsets() { - if (zkClient == null) { - error("zk client is null. Cannot commit offsets") - return + def commitOffsetToZooKeeper(topicPartition: TopicAndPartition, offset: Long) { + val topicDirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) + updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + topicPartition.partition, offset.toString) + zkCommitMeter.mark() + } + + def commitOffsets(isAutoCommit: Boolean = true) { + var retriesRemaining = 1 + (if (isAutoCommit) config.offsetsCommitMaxRetries else 0) // no retries for commits from auto-commit + var done = false + + while (!done) { + val committed = offsetsChannelLock synchronized { // committed when we receive either no error codes or only MetadataTooLarge errors + val offsetsToCommit = mutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) => + partitionTopicInfos.filterNot { case (partition, info) => + val newOffset = info.getConsumeOffset() + newOffset == checkpointedOffsets.get(TopicAndPartition(topic, info.partitionId)) + }.map { case (partition, info) => + TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset()) + } + }.toSeq:_*) + + if (offsetsToCommit.size > 0) { + if (config.offsetsStorage == "zookeeper") { + offsetsToCommit.foreach { case(topicAndPartition, offsetAndMetadata) => + commitOffsetToZooKeeper(topicAndPartition, offsetAndMetadata.offset) + checkpointedOffsets.put(topicAndPartition, offsetAndMetadata.offset) + } + true + } else { + val offsetCommitRequest = OffsetCommitRequest(config.groupId, offsetsToCommit, clientId = config.clientId) + ensureOffsetManagerConnected() + try { + kafkaCommitMeter.mark(offsetsToCommit.size) + offsetsChannel.send(offsetCommitRequest) + val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer) + trace("Offset commit response: %s.".format(offsetCommitResponse)) + + val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = { + offsetCommitResponse.commitStatus.foldLeft(false, false, false, 0) { case(folded, (topicPartition, errorCode)) => + + if (errorCode == ErrorMapping.NoError) { + val offset = offsetsToCommit(topicPartition).offset + checkpointedOffsets.put(topicPartition, offset) + if (config.dualCommitEnabled) { + commitOffsetToZooKeeper(topicPartition, offset) + } + } + + (folded._1 || // update commitFailed + errorCode != ErrorMapping.NoError, + + folded._2 || // update retryableIfFailed - (only metadata too large is not retryable) + (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.OffsetMetadataTooLargeCode), + + folded._3 || // update shouldRefreshCoordinator + errorCode == ErrorMapping.NotCoordinatorForConsumerCode || + errorCode == ErrorMapping.ConsumerCoordinatorNotAvailableCode, + + // update error count + folded._4 + (if (errorCode != ErrorMapping.NoError) 1 else 0)) + } + } + debug(errorCount + " errors in offset commit response.") + + + if (shouldRefreshCoordinator) { + debug("Could not commit offsets (because offset coordinator has moved or is unavailable).") + offsetsChannel.disconnect() + } + + if (commitFailed && retryableIfFailed) + false + else + true + } + catch { + case t: Throwable => + error("Error while committing offsets.", t) + offsetsChannel.disconnect() + false + } + } + } else { + debug("No updates to offsets since last commit.") + true + } + } + + done = if (isShuttingDown.get() && isAutoCommit) { // should not retry indefinitely if shutting down + retriesRemaining -= 1 + retriesRemaining == 0 || committed + } else + true + + if (!done) { + debug("Retrying offset commit in %d ms".format(config.offsetsChannelBackoffMs)) + Thread.sleep(config.offsetsChannelBackoffMs) + } + } + } + + private def fetchOffsetFromZooKeeper(topicPartition: TopicAndPartition) = { + val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) + val offsetString = readDataMaybeNull(zkClient, dirs.consumerOffsetDir + "/" + topicPartition.partition)._1 + offsetString match { + case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError)) + case None => (topicPartition, OffsetMetadataAndError.NoOffset) } - for ((topic, infos) <- topicRegistry) { - val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) - for (info <- infos.values) { - val newOffset = info.getConsumeOffset - if (newOffset != checkpointedOffsets.get(TopicAndPartition(topic, info.partitionId))) { + } + + private def fetchOffsets(partitions: Seq[TopicAndPartition]) = { + if (partitions.isEmpty) + Some(OffsetFetchResponse(Map.empty)) + else if (config.offsetsStorage == "zookeeper") { + val offsets = partitions.map(fetchOffsetFromZooKeeper) + Some(OffsetFetchResponse(immutable.Map(offsets:_*))) + } else { + val offsetFetchRequest = OffsetFetchRequest(groupId = config.groupId, requestInfo = partitions, clientId = config.clientId) + + var offsetFetchResponseOpt: Option[OffsetFetchResponse] = None + while (!isShuttingDown.get && !offsetFetchResponseOpt.isDefined) { + offsetFetchResponseOpt = offsetsChannelLock synchronized { + ensureOffsetManagerConnected() try { - updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partitionId, newOffset.toString) - checkpointedOffsets.put(TopicAndPartition(topic, info.partitionId), newOffset) - } catch { - case t: Throwable => - // log it and let it go - warn("exception during commitOffsets", t) + offsetsChannel.send(offsetFetchRequest) + val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().buffer) + trace("Offset fetch response: %s.".format(offsetFetchResponse)) + + val (leaderChanged, loadInProgress) = + offsetFetchResponse.requestInfo.foldLeft(false, false) { case(folded, (topicPartition, offsetMetadataAndError)) => + (folded._1 || (offsetMetadataAndError.error == ErrorMapping.NotCoordinatorForConsumerCode), + folded._2 || (offsetMetadataAndError.error == ErrorMapping.OffsetsLoadInProgressCode)) + } + + if (leaderChanged) { + offsetsChannel.disconnect() + debug("Could not fetch offsets (because offset manager has moved).") + None // retry + } + else if (loadInProgress) { + debug("Could not fetch offsets (because offset cache is being loaded).") + None // retry + } + else { + if (config.dualCommitEnabled) { + // if dual-commit is enabled (i.e., if a consumer group is migrating offsets to kafka), then pick the + // maximum between offsets in zookeeper and kafka. + val kafkaOffsets = offsetFetchResponse.requestInfo + val mostRecentOffsets = kafkaOffsets.map { case (topicPartition, kafkaOffset) => + val zkOffset = fetchOffsetFromZooKeeper(topicPartition)._2.offset + val mostRecentOffset = zkOffset.max(kafkaOffset.offset) + (topicPartition, OffsetMetadataAndError(mostRecentOffset, kafkaOffset.metadata, ErrorMapping.NoError)) + } + Some(OffsetFetchResponse(mostRecentOffsets)) + } + else + Some(offsetFetchResponse) + } + } + catch { + case e: Exception => + error("Error while fetching offsets from %s:%d.".format(offsetsChannel.host, offsetsChannel.port), e) + offsetsChannel.disconnect() + None // retry } - debug("Committed offset " + newOffset + " for topic " + info) + } + + if (offsetFetchResponseOpt.isEmpty) { + debug("Retrying offset fetch in %d ms".format(config.offsetsChannelBackoffMs)) + Thread.sleep(config.offsetsChannelBackoffMs) } } + + offsetFetchResponseOpt } } + class ZKSessionExpireListener(val dirs: ZKGroupDirs, val consumerIdString: String, val topicCount: TopicCount, @@ -433,8 +612,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def rebalance(cluster: Cluster): Boolean = { - val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic - val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) + val myTopicThreadIdsMap = TopicCount.constructTopicCount( + group, consumerIdString, zkClient, config.excludeInternalTopics).getConsumerThreadIdsPerTopic + val consumersPerTopicMap = getConsumersPerTopic(zkClient, group, config.excludeInternalTopics) val brokers = getAllBrokersInCluster(zkClient) if (brokers.size == 0) { // This can happen in a rare case when there are no brokers available in the cluster when the consumer is started. @@ -458,13 +638,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, releasePartitionOwnership(topicRegistry) - var partitionOwnershipDecision = new collection.mutable.HashMap[(String, Int), String]() + var partitionOwnershipDecision = new collection.mutable.HashMap[TopicAndPartition, String]() val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) { currentTopicRegistry.put(topic, new Pool[Int, PartitionTopicInfo]) - val topicDirs = new ZKGroupTopicDirs(group, topic) val curConsumers = consumersPerTopicMap.get(topic).get val curPartitions: Seq[Int] = partitionsPerTopicMap.get(topic).get @@ -490,27 +669,42 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, for (i <- startPart until startPart + nParts) { val partition = curPartitions(i) info(consumerThreadId + " attempting to claim partition " + partition) - addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId) // record the partition ownership decision - partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) + partitionOwnershipDecision += (TopicAndPartition(topic, partition) -> consumerThreadId) } } } } - /** - * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt - * A rebalancing attempt is completed successfully only after the fetchers have been started correctly - */ - if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { - info("Updating the cache") - debug("Partitions per topic cache " + partitionsPerTopicMap) - debug("Consumers per topic cache " + consumersPerTopicMap) - topicRegistry = currentTopicRegistry - updateFetcher(cluster) - true - } else { + // fetch current offsets for all topic-partitions + val topicPartitions = partitionOwnershipDecision.keySet.toSeq + val offsetFetchResponseOpt = fetchOffsets(topicPartitions) + + if (isShuttingDown.get || !offsetFetchResponseOpt.isDefined) false + else { + val offsetFetchResponse = offsetFetchResponseOpt.get + topicPartitions.foreach { topicAndPartition => + val (topic, partition) = topicAndPartition.asTuple + val offset = offsetFetchResponse.requestInfo(topicAndPartition).offset + val threadId = partitionOwnershipDecision(topicAndPartition) + addPartitionTopicInfo(currentTopicRegistry, partition, topic, offset, threadId) + } + + /** + * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt + * A rebalancing attempt is completed successfully only after the fetchers have been started correctly + */ + if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { + info("Updating the cache") + debug("Partitions per topic cache " + partitionsPerTopicMap) + debug("Consumers per topic cache " + consumersPerTopicMap) + topicRegistry = currentTopicRegistry + updateFetcher(cluster) + true + } else { + false + } } } } @@ -533,7 +727,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * successfully and the fetchers restart to fetch more data chunks **/ if (config.autoCommitEnable) - commitOffsets + commitOffsets() case None => } } @@ -578,11 +772,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, Int), String]): Boolean = { + private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[TopicAndPartition, String]): Boolean = { var successfullyOwnedPartitions : List[(String, Int)] = Nil val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => - val topic = partitionOwner._1._1 - val partition = partitionOwner._1._2 + val topic = partitionOwner._1.topic + val partition = partitionOwner._1.partition val consumerThreadId = partitionOwner._2 val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition) try { @@ -609,18 +803,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]], - topicDirs: ZKGroupTopicDirs, partition: Int, - topic: String, consumerThreadId: String) { + partition: Int, topic: String, + offset: Long, consumerThreadId: String) { val partTopicInfoMap = currentTopicRegistry.get(topic) - val znode = topicDirs.consumerOffsetDir + "/" + partition - val offsetString = readDataMaybeNull(zkClient, znode)._1 - // If first time starting a consumer, set the initial offset to -1 - val offset = - offsetString match { - case Some(offsetStr) => offsetStr.toLong - case None => PartitionTopicInfo.InvalidOffset - } val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) val consumedOffset = new AtomicLong(offset) val fetchedOffset = new AtomicLong(offset) @@ -746,10 +932,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, // bootstrap with existing topics private var wildcardTopics = getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) - .filter(topicFilter.isTopicAllowed) + .filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics)) private val wildcardTopicCount = TopicCount.constructTopicCount( - consumerIdString, topicFilter, numStreams, zkClient) + consumerIdString, topicFilter, numStreams, zkClient, config.excludeInternalTopics) val dirs = new ZKGroupDirs(config.groupId) registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount) @@ -764,7 +950,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def handleTopicEvent(allTopics: Seq[String]) { debug("Handling topic event") - val updatedTopics = allTopics.filter(topicFilter.isTopicAllowed) + val updatedTopics = allTopics.filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics)) val addedTopics = updatedTopics filterNot (wildcardTopics contains) if (addedTopics.nonEmpty) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 4deff9db9d63..5db24a73a62c 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -604,7 +604,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def onPreferredReplicaElection(partitions: Set[TopicAndPartition], isTriggeredByAutoRebalance: Boolean = true) { + def onPreferredReplicaElection(partitions: Set[TopicAndPartition], isTriggeredByAutoRebalance: Boolean = false) { info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions @@ -1116,7 +1116,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg controllerContext.partitionsUndergoingPreferredReplicaElection.size == 0 && !deleteTopicManager.isTopicQueuedUpForDeletion(topicPartition.topic) && controllerContext.allTopics.contains(topicPartition.topic)) { - onPreferredReplicaElection(Set(topicPartition), false) + onPreferredReplicaElection(Set(topicPartition), true) } } } diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala new file mode 100644 index 000000000000..dfa9c42bcab3 --- /dev/null +++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala @@ -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 kafka.javaapi + +import kafka.cluster.Broker + +class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) { + + def errorCode = underlying.errorCode + + def coordinator: Broker = { + import kafka.javaapi.Implicits._ + underlying.coordinator + } + + override def equals(other: Any) = canEqual(other) && { + val otherConsumerMetadataResponse = other.asInstanceOf[kafka.javaapi.ConsumerMetadataResponse] + this.underlying.equals(otherConsumerMetadataResponse.underlying) + } + + def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.ConsumerMetadataResponse] + + override def hashCode = underlying.hashCode + + override def toString = underlying.toString + +} diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala index 57b9d2a9c197..6de320dcc0d9 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala @@ -17,19 +17,18 @@ package kafka.javaapi -import kafka.common.{TopicAndPartition, OffsetMetadataAndError} -import collection.JavaConversions -import java.nio.ByteBuffer +import kafka.common.{OffsetAndMetadata, TopicAndPartition} class OffsetCommitRequest(groupId: String, - requestInfo: java.util.Map[TopicAndPartition, OffsetMetadataAndError], + requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], versionId: Short, correlationId: Int, clientId: String) { val underlying = { - val scalaMap: Map[TopicAndPartition, OffsetMetadataAndError] = { - import JavaConversions._ - requestInfo.toMap + val scalaMap: collection.mutable.Map[TopicAndPartition, OffsetAndMetadata] = { + import collection.JavaConversions._ + + collection.JavaConversions.asMap(requestInfo) } kafka.api.OffsetCommitRequest( groupId = groupId, diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala index 570bf3173f48..c2d3d114b82a 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala @@ -24,7 +24,7 @@ class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitRespons def errors: java.util.Map[TopicAndPartition, Short] = { import JavaConversions._ - underlying.requestInfo + underlying.commitStatus } } diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index c45c80399c21..44d3d35c64c3 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -18,13 +18,12 @@ package kafka.javaapi.consumer; +import java.util.List; +import java.util.Map; import kafka.consumer.KafkaStream; import kafka.consumer.TopicFilter; import kafka.serializer.Decoder; -import java.util.List; -import java.util.Map; - public interface ConsumerConnector { /** * Create a list of MessageStreams of type T for each topic. @@ -62,6 +61,7 @@ public interface ConsumerConnector { * Commit the offsets of all broker partitions connected by this connector. */ public void commitOffsets(); + public void commitOffsets(boolean retryOnFailure); /** * Shut down the connector diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala index 58e83f6119d3..1f95d9b6a560 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala @@ -101,9 +101,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def createMessageStreamsByFilter(topicFilter: TopicFilter) = createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder()) - + def commitOffsets() { - underlying.commitOffsets + underlying.commitOffsets() + } + + def commitOffsets(retryOnFailure: Boolean) { + underlying.commitOffsets(retryOnFailure) } def shutdown() { diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index e1f8b979c3e6..b2652ddbe2f8 100644 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -255,8 +255,8 @@ class FileMessageSet private[kafka](@volatile var file: File, /** * Read from the underlying file into the buffer starting at the given position */ - def readInto(buffer: ByteBuffer, position: Int): ByteBuffer = { - channel.read(buffer, position) + def readInto(buffer: ByteBuffer, relativePosition: Int): ByteBuffer = { + channel.read(buffer, relativePosition + this.start) buffer.flip() buffer } diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index 7947b18aceb2..3cdf23dce340 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -77,16 +77,7 @@ class ProducerConfig private (val props: VerifiableProperties) * This parameter allows you to specify the compression codec for all data generated * * by this producer. The default is NoCompressionCodec */ - val compressionCodec = { - val prop = props.getString("compression.codec", NoCompressionCodec.name) - try { - CompressionCodec.getCompressionCodec(prop.toInt) - } - catch { - case nfe: NumberFormatException => - CompressionCodec.getCompressionCodec(prop) - } - } + val compressionCodec = props.getCompressionCodec("compression.codec", NoCompressionCodec) /** This parameter allows you to set whether compression should be turned * * on for particular topics diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 215ac36ece44..1d9922b2ef8e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -22,24 +22,23 @@ import kafka.api._ import kafka.message._ import kafka.network._ import kafka.log._ -import kafka.utils.ZKGroupTopicDirs import scala.collection._ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic._ import kafka.metrics.KafkaMetricsGroup -import org.I0Itec.zkclient.ZkClient import kafka.common._ -import kafka.utils.{ZkUtils, Pool, SystemTime, Logging} +import kafka.utils.{Pool, SystemTime, Logging} import kafka.network.RequestChannel.Response import kafka.cluster.Broker import kafka.controller.KafkaController - +import org.I0Itec.zkclient.ZkClient /** * Logic to handle the various Kafka requests */ class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, + val offsetManager: OffsetManager, val zkClient: ZkClient, val brokerId: Int, val config: KafkaConfig, @@ -65,7 +64,7 @@ class KafkaApis(val requestChannel: RequestChannel, try{ trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) request.requestId match { - case RequestKeys.ProduceKey => handleProducerRequest(request) + case RequestKeys.ProduceKey => handleProducerOrOffsetCommitRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) case RequestKeys.OffsetsKey => handleOffsetRequest(request) case RequestKeys.MetadataKey => handleTopicMetadataRequest(request) @@ -73,8 +72,9 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request) case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request) case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request) - case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) + case RequestKeys.OffsetCommitKey => handleProducerOrOffsetCommitRequest(request) case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) + case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) } } catch { @@ -97,7 +97,7 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val leaderAndIsrRequest = request.requestObj.asInstanceOf[LeaderAndIsrRequest] try { - val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest) + val (response, error) = replicaManager.becomeLeaderOrFollower(leaderAndIsrRequest, offsetManager) val leaderAndIsrResponse = new LeaderAndIsrResponse(leaderAndIsrRequest.correlationId, response, error) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(leaderAndIsrResponse))) } catch { @@ -192,14 +192,47 @@ class KafkaApis(val requestChannel: RequestChannel, } } + private def producerRequestFromOffsetCommit(offsetCommitRequest: OffsetCommitRequest) = { + val msgs = offsetCommitRequest.filterLargeMetadata(config.offsetMetadataMaxSize).map { + case (topicAndPartition, offset) => + new Message( + bytes = OffsetManager.offsetCommitValue(offset), + key = OffsetManager.offsetCommitKey(offsetCommitRequest.groupId, topicAndPartition.topic, topicAndPartition.partition) + ) + }.toSeq + + val producerData = mutable.Map( + TopicAndPartition(OffsetManager.OffsetsTopicName, offsetManager.partitionFor(offsetCommitRequest.groupId)) -> + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, msgs:_*) + ) + + val request = ProducerRequest( + correlationId = offsetCommitRequest.correlationId, + clientId = offsetCommitRequest.clientId, + requiredAcks = config.offsetCommitRequiredAcks, + ackTimeoutMs = config.offsetCommitTimeoutMs, + data = producerData) + trace("Created producer request %s for offset commit request %s.".format(request, offsetCommitRequest)) + request + } + /** - * Handle a produce request + * Handle a produce request or offset commit request (which is really a specialized producer request) */ - def handleProducerRequest(request: RequestChannel.Request) { - val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] + def handleProducerOrOffsetCommitRequest(request: RequestChannel.Request) { + + val (produceRequest, offsetCommitRequestOpt) = if (request.requestId == RequestKeys.OffsetCommitKey) { + val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + (producerRequestFromOffsetCommit(offsetCommitRequest), Some(offsetCommitRequest)) + } + else { + (request.requestObj.asInstanceOf[ProducerRequest], None) + } + val sTime = SystemTime.milliseconds val localProduceResults = appendToLocalLog(produceRequest) debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) + val firstErrorCode = localProduceResults.find(_.errorCode != ErrorMapping.NoError).map(_.errorCode).getOrElse(ErrorMapping.NoError) val numPartitionsInError = localProduceResults.count(_.error.isDefined) produceRequest.data.foreach(partitionAndData => @@ -218,14 +251,29 @@ class KafkaApis(val requestChannel: RequestChannel, .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) requestChannel.closeConnection(request.processor, request) } else { - requestChannel.noOperation(request.processor, request) + + if (firstErrorCode == ErrorMapping.NoError) + offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo)) + + if (offsetCommitRequestOpt.isDefined) { + val response = offsetCommitRequestOpt.get.responseFor(firstErrorCode, config.offsetMetadataMaxSize) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } else + requestChannel.noOperation(request.processor, request) } } else if (produceRequest.requiredAcks == 1 || produceRequest.numPartitions <= 0 || allPartitionHaveReplicationFactorOne || numPartitionsInError == produceRequest.numPartitions) { + + if (firstErrorCode == ErrorMapping.NoError) { + offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) + } + val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap - val response = ProducerResponse(produceRequest.correlationId, statuses) + val response = offsetCommitRequestOpt.map(_.responseFor(firstErrorCode, config.offsetMetadataMaxSize)) + .getOrElse(ProducerResponse(produceRequest.correlationId, statuses)) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } else { // create a list of (topic, partition) pairs to use as keys for this delayed request @@ -233,12 +281,15 @@ class KafkaApis(val requestChannel: RequestChannel, topicAndPartition => new RequestKey(topicAndPartition)).toSeq val statuses = localProduceResults.map(r => r.key -> DelayedProduceResponseStatus(r.end + 1, ProducerResponseStatus(r.errorCode, r.start))).toMap - val delayedProduce = new DelayedProduce(producerRequestKeys, - request, - statuses, - produceRequest, - produceRequest.ackTimeoutMs.toLong) - producerRequestPurgatory.watch(delayedProduce) + val delayedRequest = new DelayedProduce( + producerRequestKeys, + request, + statuses, + produceRequest, + produceRequest.ackTimeoutMs.toLong, + offsetCommitRequestOpt) + + producerRequestPurgatory.watch(delayedRequest) /* * Replica fetch requests may have arrived (and potentially satisfied) @@ -252,6 +303,7 @@ class KafkaApis(val requestChannel: RequestChannel, debug(satisfiedProduceRequests.size + " producer requests unblocked during produce to local log.") satisfiedProduceRequests.foreach(_.respond()) + // we do not need the data anymore produceRequest.emptyData() } @@ -265,7 +317,7 @@ class KafkaApis(val requestChannel: RequestChannel, "acksPending:%b, error: %d, startOffset: %d, requiredOffset: %d".format( acksPending, status.error, status.offset, requiredOffset) } - + case class ProduceResult(key: TopicAndPartition, start: Long, end: Long, error: Option[Throwable] = None) { def this(key: TopicAndPartition, throwable: Throwable) = this(key, -1L, -1L, Some(throwable)) @@ -557,120 +609,85 @@ class KafkaApis(val requestChannel: RequestChannel, ret.toSeq.sortBy(- _) } - /** - * Service the topic metadata request API - */ - def handleTopicMetadataRequest(request: RequestChannel.Request) { - val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - val topicsMetadata = new mutable.ArrayBuffer[TopicMetadata]() + private def getTopicMetadata(topics: Set[String]): Seq[TopicMetadata] = { val config = replicaManager.config - var uniqueTopics = Set.empty[String] - uniqueTopics = { - if(metadataRequest.topics.size > 0) - metadataRequest.topics.toSet - else { - partitionMetadataLock synchronized { - metadataCache.keySet.map(_.topic) - } - } - } - val topicMetadataList = - partitionMetadataLock synchronized { - uniqueTopics.map { topic => - if(metadataCache.keySet.map(_.topic).contains(topic)) { - debug("Topic %s exists in metadata cache on broker %d".format(topic, config.brokerId)) - val partitionStateInfo = metadataCache.filter(p => p._1.topic.equals(topic)) - val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition) - val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) => - val replicas = metadataCache(topicAndPartition).allReplicas - var replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq - var leaderInfo: Option[Broker] = None - var isrInfo: Seq[Broker] = Nil - val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch - val leader = leaderIsrAndEpoch.leaderAndIsr.leader - val isr = leaderIsrAndEpoch.leaderAndIsr.isr - debug("%s".format(topicAndPartition) + ";replicas = " + replicas + ", in sync replicas = " + isr + ", leader = " + leader) - try { - if(aliveBrokers.keySet.contains(leader)) - leaderInfo = Some(aliveBrokers(leader)) - else throw new LeaderNotAvailableException("Leader not available for partition %s".format(topicAndPartition)) - isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null) - if(replicaInfo.size < replicas.size) - throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + - replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) - if(isrInfo.size < isr.size) - throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + - isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) - new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) - } catch { - case e: Throwable => - error("Error while fetching metadata for partition %s".format(topicAndPartition), e) - new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, - ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - } + + partitionMetadataLock synchronized { + topics.map { topic => + if(metadataCache.keySet.map(_.topic).contains(topic)) { + val partitionStateInfo = metadataCache.filter(p => p._1.topic.equals(topic)) + val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition) + val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) => + val replicas = metadataCache(topicAndPartition).allReplicas + val replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq + var leaderInfo: Option[Broker] = None + var isrInfo: Seq[Broker] = Nil + val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch + val leader = leaderIsrAndEpoch.leaderAndIsr.leader + val isr = leaderIsrAndEpoch.leaderAndIsr.isr + debug("%s".format(topicAndPartition) + ";replicas = " + replicas + ", in sync replicas = " + isr + ", leader = " + leader) + try { + if(aliveBrokers.keySet.contains(leader)) + leaderInfo = Some(aliveBrokers(leader)) + else throw new LeaderNotAvailableException("Leader not available for partition %s".format(topicAndPartition)) + isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null) + if(replicaInfo.size < replicas.size) + throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + + replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) + if(isrInfo.size < isr.size) + throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + + isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) + new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) + } catch { + case e: Throwable => + error("Error while fetching metadata for partition %s".format(topicAndPartition), e) + new PartitionMetadata(topicAndPartition.partition, leaderInfo, replicaInfo, isrInfo, + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } - new TopicMetadata(topic, partitionMetadata) - } else { - debug("Topic %s does not exist in metadata cache on broker %d".format(topic, config.brokerId)) - // topic doesn't exist, send appropriate error code - new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) } - } - } - - // handle auto create topics - topicMetadataList.foreach { topicMetadata => - topicMetadata.errorCode match { - case ErrorMapping.NoError => topicsMetadata += topicMetadata - case ErrorMapping.UnknownTopicOrPartitionCode => - if (config.autoCreateTopicsEnable) { + new TopicMetadata(topic, partitionMetadata) + } else { + // topic doesn't exist, send appropriate error code after handling auto create topics + val isOffsetsTopic = topic == OffsetManager.OffsetsTopicName + if (config.autoCreateTopicsEnable || isOffsetsTopic) { try { - AdminUtils.createTopic(zkClient, topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor) + if (isOffsetsTopic) + AdminUtils.createTopic(zkClient, topic, + config.offsetsTopicPartitions, config.offsetsTopicReplicationFactor, offsetManager.offsetsTopicConfig) + else + AdminUtils.createTopic(zkClient, topic, config.numPartitions, config.defaultReplicationFactor) info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" - .format(topicMetadata.topic, config.numPartitions, config.defaultReplicationFactor)) + .format(topic, config.numPartitions, config.defaultReplicationFactor)) } catch { case e: TopicExistsException => // let it go, possibly another broker created this topic } - topicsMetadata += new TopicMetadata(topicMetadata.topic, topicMetadata.partitionsMetadata, ErrorMapping.LeaderNotAvailableCode) + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.LeaderNotAvailableCode) } else { - debug("Auto create topic skipped for %s".format(topicMetadata.topic)) - topicsMetadata += topicMetadata + new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) } - case _ => - debug("Error while fetching topic metadata for topic %s due to %s ".format(topicMetadata.topic, - ErrorMapping.exceptionFor(topicMetadata.errorCode).getClass.getName)) - topicsMetadata += topicMetadata + } } - } - trace("Sending topic metadata %s for correlation id %d to client %s".format(topicsMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) - val response = new TopicMetadataResponse(topicsMetadata.toSeq, metadataRequest.correlationId) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + }.toSeq } - /* - * Service the Offset commit API + /** + * Service the topic metadata request API */ - def handleOffsetCommitRequest(request: RequestChannel.Request) { - val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - val responseInfo = offsetCommitRequest.requestInfo.map{ - case (topicAndPartition, metaAndError) => { - val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicAndPartition.topic) - try { - ensureTopicExists(topicAndPartition.topic) - if(metaAndError.metadata != null && metaAndError.metadata.length > config.offsetMetadataMaxSize) { - (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) - } else { - ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + - topicAndPartition.partition, metaAndError.offset.toString) - (topicAndPartition, ErrorMapping.NoError) - } - } catch { - case e: Throwable => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + def handleTopicMetadataRequest(request: RequestChannel.Request) { + val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] + var uniqueTopics = Set.empty[String] + uniqueTopics = { + if(metadataRequest.topics.size > 0) + metadataRequest.topics.toSet + else { + partitionMetadataLock synchronized { + metadataCache.keySet.map(_.topic) } } } - val response = new OffsetCommitResponse(responseInfo, - offsetCommitRequest.correlationId) + val topicMetadata = getTopicMetadata(uniqueTopics) + trace("Sending topic metadata %s for correlation id %d to client %s".format(topicMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) + val response = new TopicMetadataResponse(topicMetadata, metadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -679,26 +696,38 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] - val responseInfo = offsetFetchRequest.requestInfo.map( t => { - val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, t.topic) - try { - ensureTopicExists(t.topic) - val payloadOpt = ZkUtils.readDataMaybeNull(zkClient, topicDirs.consumerOffsetDir + "/" + t.partition)._1 - payloadOpt match { - case Some(payload) => { - (t, OffsetMetadataAndError(offset=payload.toLong, error=ErrorMapping.NoError)) - } - case None => (t, OffsetMetadataAndError(OffsetMetadataAndError.InvalidOffset, OffsetMetadataAndError.NoMetadata, - ErrorMapping.UnknownTopicOrPartitionCode)) - } - } catch { - case e: Throwable => - (t, OffsetMetadataAndError(OffsetMetadataAndError.InvalidOffset, OffsetMetadataAndError.NoMetadata, - ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))) - } - }) - val response = new OffsetFetchResponse(collection.immutable.Map(responseInfo: _*), - offsetFetchRequest.correlationId) + + val status = offsetManager.getOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap + + val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId) + + trace("Sending offset fetch response %s for correlation id %d to client %s." + .format(response, offsetFetchRequest.correlationId, offsetFetchRequest.clientId)) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } + + /* + * Service the consumer metadata API + */ + def handleConsumerMetadataRequest(request: RequestChannel.Request) { + val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest] + + val partition = offsetManager.partitionFor(consumerMetadataRequest.group) + + // get metadata (and create the topic if necessary) + val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head + + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) + + val response = + offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata => + partitionMetadata.leader.map { leader => + ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId) + }.getOrElse(errorResponse) + }.getOrElse(errorResponse) + + trace("Sending consumer metadata %s for correlation id %d to client %s." + .format(response, consumerMetadataRequest.correlationId, consumerMetadataRequest.clientId)) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -772,12 +801,13 @@ class KafkaApis(val requestChannel: RequestChannel, class DelayedProduce(keys: Seq[RequestKey], request: RequestChannel.Request, - val partitionStatus: Map[TopicAndPartition, DelayedProduceResponseStatus], - val produce: ProducerRequest, - delayMs: Long) + val partitionStatus: immutable.Map[TopicAndPartition, DelayedProduceResponseStatus], + produce: ProducerRequest, + delayMs: Long, + offsetCommitRequestOpt: Option[OffsetCommitRequest] = None) extends DelayedRequest(keys, request, delayMs) with Logging { - // first update the acks pending variable according to error code + // first update the acks pending variable according to the error code partitionStatus foreach { case (topicAndPartition, delayedStatus) => if (delayedStatus.status.error == ErrorMapping.NoError) { // Timeout error state will be cleared when requiredAcks are received @@ -790,13 +820,21 @@ class KafkaApis(val requestChannel: RequestChannel, trace("Initial partition status for %s is %s".format(topicAndPartition, delayedStatus)) } - def respond() { val responseStatus = partitionStatus.map { case (topicAndPartition, delayedStatus) => topicAndPartition -> delayedStatus.status } - val response = ProducerResponse(produce.correlationId, responseStatus) + val errorCode = responseStatus.find { case (_, status) => + status.error != ErrorMapping.NoError + }.map(_._2.error).getOrElse(ErrorMapping.NoError) + + if (errorCode == ErrorMapping.NoError) { + offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) + } + + val response = offsetCommitRequestOpt.map(_.responseFor(errorCode, config.offsetMetadataMaxSize)) + .getOrElse(ProducerResponse(produce.correlationId, responseStatus)) requestChannel.sendResponse(new RequestChannel.Response( request, new BoundedByteBufferSend(response))) @@ -828,7 +866,7 @@ class KafkaApis(val requestChannel: RequestChannel, (false, ErrorMapping.UnknownTopicOrPartitionCode) } if (errorCode != ErrorMapping.NoError) { - fetchPartitionStatus.acksPending = false + fetchPartitionStatus. acksPending = false fetchPartitionStatus.status.error = errorCode } else if (hasEnough) { fetchPartitionStatus.acksPending = false diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index b871843c2182..d07796ec87fa 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -58,7 +58,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val numIoThreads = props.getIntInRange("num.io.threads", 8, (1, Int.MaxValue)) /* the number of threads to use for various background processing tasks */ - val backgroundThreads = props.getIntInRange("background.threads", 4, (1, Int.MaxValue)) + val backgroundThreads = props.getIntInRange("background.threads", 10, (1, Int.MaxValue)) /* the number of queued requests allowed before blocking the network threads */ val queuedMaxRequests = props.getIntInRange("queued.max.requests", 500, (1, Int.MaxValue)) @@ -242,10 +242,46 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val controlledShutdownRetryBackoffMs = props.getInt("controlled.shutdown.retry.backoff.ms", 5000) /* enable controlled shutdown of the server */ - val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", false) + val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", default = false) - /*********** Misc configuration ***********/ + /*********** Offset management configuration ***********/ /* the maximum size for a metadata entry associated with an offset commit */ - val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", 1024) + val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", OffsetManagerConfig.DefaultMaxMetadataSize) + + /** Batch size for reading from the offsets segments when loading offsets into the cache. */ + val offsetsLoadBufferSize = props.getIntInRange("offsets.load.buffer.size", + OffsetManagerConfig.DefaultLoadBufferSize, (1, Integer.MAX_VALUE)) + + /** The replication factor for the offset commit topic (set higher to ensure availability). */ + val offsetsTopicReplicationFactor: Short = props.getShortInRange("offsets.topic.replication.factor", + OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor, (1, Short.MaxValue)) + + /** The number of partitions for the offset commit topic (should not change after deployment). */ + val offsetsTopicPartitions: Int = props.getIntInRange("offsets.topic.num.partitions", + OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, (1, Integer.MAX_VALUE)) + + /** The offsets topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads */ + val offsetsTopicSegmentBytes: Int = props.getIntInRange("offsets.topic.segment.bytes", + OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, (1, Integer.MAX_VALUE)) + + /** Compression codec for the offsets topic - compression may be used to achieve "atomic" commits. */ + val offsetsTopicCompressionCodec = props.getCompressionCodec("offsets.topic.compression.codec", + OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec) + + /** Offsets older than this retention period will be discarded. */ + val offsetsRetentionMinutes: Int = props.getIntInRange("offsets.retention.minutes", 24*60, (1, Integer.MAX_VALUE)) + + /** Frequency at which to check for stale offsets. */ + val offsetsRetentionCheckIntervalMs: Long = props.getLongInRange("offsets.retention.check.interval.ms", + OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, (1, Long.MaxValue)) + + /* Offset commit will be delayed until all replicas for the offsets topic receive the commit or this timeout is + * reached. This is similar to the producer request timeout. */ + val offsetCommitTimeoutMs = props.getIntInRange("offsets.commit.timeout.ms", + OffsetManagerConfig.DefaultOffsetCommitTimeoutMs, (1, Integer.MAX_VALUE)) + + /** The required acks before the commit can be accepted. In general, the default (-1) should not be overridden. */ + val offsetCommitRequiredAcks = props.getShortInRange("offsets.commit.required.acks", + OffsetManagerConfig.DefaultOffsetCommitRequiredAcks, (-1, offsetsTopicReplicationFactor)) } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index feb2093e6897..c208f83bed7f 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -40,11 +40,12 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg this.logIdent = "[Kafka Server " + config.brokerId + "], " private var isShuttingDown = new AtomicBoolean(false) private var shutdownLatch = new CountDownLatch(1) - private var startupComplete = new AtomicBoolean(false); + private var startupComplete = new AtomicBoolean(false) val correlationId: AtomicInteger = new AtomicInteger(0) var socketServer: SocketServer = null var requestHandlerPool: KafkaRequestHandlerPool = null var logManager: LogManager = null + var offsetManager: OffsetManager = null var kafkaHealthcheck: KafkaHealthcheck = null var topicConfigManager: TopicConfigManager = null var replicaManager: ReplicaManager = null @@ -83,10 +84,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg socketServer.startup() replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) + + /* start offset manager */ + offsetManager = createOffsetManager() + kafkaController = new KafkaController(config, zkClient) /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, zkClient, config.brokerId, config, kafkaController) + apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, zkClient, config.brokerId, config, kafkaController) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) Mx4jLoader.maybeLoad() @@ -104,7 +109,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg registerStats() - startupComplete.set(true); + startupComplete.set(true) info("started") } @@ -215,7 +220,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg */ def shutdown() { info("shutting down") - val canShutdown = isShuttingDown.compareAndSet(false, true); + val canShutdown = isShuttingDown.compareAndSet(false, true) if (canShutdown) { Utils.swallow(controlledShutdown()) if(kafkaHealthcheck != null) @@ -224,6 +229,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Utils.swallow(socketServer.shutdown()) if(requestHandlerPool != null) Utils.swallow(requestHandlerPool.shutdown()) + if(offsetManager != null) + offsetManager.shutdown() Utils.swallow(kafkaScheduler.shutdown()) if(apis != null) Utils.swallow(apis.close()) @@ -237,7 +244,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Utils.swallow(zkClient.close()) shutdownLatch.countDown() - startupComplete.set(false); + startupComplete.set(false) info("shut down completed") } } @@ -285,6 +292,17 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg time = time) } -} + private def createOffsetManager(): OffsetManager = { + val offsetManagerConfig = OffsetManagerConfig( + maxMetadataSize = config.offsetMetadataMaxSize, + loadBufferSize = config.offsetsLoadBufferSize, + offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsTopicNumPartitions = config.offsetsTopicPartitions, + offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, + offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, + offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) + new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler) + } +} diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala new file mode 100644 index 000000000000..89a88a7e4055 --- /dev/null +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -0,0 +1,480 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.utils._ +import kafka.common._ +import java.nio.ByteBuffer +import java.util.Properties +import kafka.log.{FileMessageSet, LogConfig} +import org.I0Itec.zkclient.ZkClient +import scala.collection._ +import kafka.message._ +import java.util.concurrent.TimeUnit +import kafka.metrics.KafkaMetricsGroup +import com.yammer.metrics.core.Gauge +import scala.Some +import kafka.common.TopicAndPartition +import kafka.consumer.MessageFormatter +import java.io.PrintStream +import org.apache.kafka.common.protocol.types.{Struct, Schema, Field} +import org.apache.kafka.common.protocol.types.Type.STRING +import org.apache.kafka.common.protocol.types.Type.INT32 +import org.apache.kafka.common.protocol.types.Type.INT64 +import java.util.concurrent.atomic.AtomicBoolean + + +/** + * Configuration settings for in-built offset management + * @param maxMetadataSize The maximum allowed metadata for any offset commit. + * @param loadBufferSize Batch size for reading from the offsets segments when loading offsets into the cache. + * @param offsetsRetentionMs Offsets older than this retention period will be discarded. + * @param offsetsRetentionCheckIntervalMs Frequency at which to check for stale offsets. + * @param offsetsTopicNumPartitions The number of partitions for the offset commit topic (should not change after deployment). + * @param offsetsTopicSegmentBytes The offsets topic segment bytes should be kept relatively small to facilitate faster + * log compaction and faster offset loads + * @param offsetsTopicReplicationFactor The replication factor for the offset commit topic (set higher to ensure availability). + * @param offsetsTopicCompressionCodec Compression codec for the offsets topic - compression should be turned on in + * order to achieve "atomic" commits. + * @param offsetCommitTimeoutMs The offset commit will be delayed until all replicas for the offsets topic receive the + * commit or this timeout is reached. (Similar to the producer request timeout.) + * @param offsetCommitRequiredAcks The required acks before the commit can be accepted. In general, the default (-1) + * should not be overridden. + */ +case class OffsetManagerConfig(maxMetadataSize: Int = OffsetManagerConfig.DefaultMaxMetadataSize, + loadBufferSize: Int = OffsetManagerConfig.DefaultLoadBufferSize, + offsetsRetentionMs: Long = 24*60*60000L, + offsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, + offsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, + offsetsTopicReplicationFactor: Short = OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor, + offsetsTopicCompressionCodec: CompressionCodec = OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec, + offsetCommitTimeoutMs: Int = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs, + offsetCommitRequiredAcks: Short = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks) + +object OffsetManagerConfig { + val DefaultMaxMetadataSize = 4096 + val DefaultLoadBufferSize = 5*1024*1024 + val DefaultOffsetsRetentionCheckIntervalMs = 600000L + val DefaultOffsetsTopicNumPartitions = 1 + val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 + val DefaultOffsetsTopicReplicationFactor = 1.toShort + val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec + val DefaultOffsetCommitTimeoutMs = 5000 + val DefaultOffsetCommitRequiredAcks = (-1).toShort +} + +class OffsetManager(val config: OffsetManagerConfig, + replicaManager: ReplicaManager, + zkClient: ZkClient, + scheduler: Scheduler) extends Logging with KafkaMetricsGroup { + + /* offsets and metadata cache */ + private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata] + private val followerTransitionLock = new Object + + private val loadingPartitions: mutable.Set[Int] = mutable.Set() + + private val shuttingDown = new AtomicBoolean(false) + + scheduler.schedule(name = "offsets-cache-compactor", + fun = compact, + period = config.offsetsRetentionCheckIntervalMs, + unit = TimeUnit.MILLISECONDS) + + newGauge("NumOffsets", + new Gauge[Int] { + def value = offsetsCache.size + } + ) + + newGauge("NumGroups", + new Gauge[Int] { + def value = offsetsCache.keys.map(_.group).toSet.size + } + ) + + private def compact() { + debug("Compacting offsets cache.") + val startMs = SystemTime.milliseconds + + val staleOffsets = offsetsCache.filter(startMs - _._2.timestamp > config.offsetsRetentionMs) + + debug("Found %d stale offsets (older than %d ms).".format(staleOffsets.size, config.offsetsRetentionMs)) + + // delete the stale offsets from the table and generate tombstone messages to remove them from the log + val tombstonesForPartition = staleOffsets.map { case(groupTopicAndPartition, offsetAndMetadata) => + val offsetsPartition = partitionFor(groupTopicAndPartition.group) + trace("Removing stale offset and metadata for %s: %s".format(groupTopicAndPartition, offsetAndMetadata)) + + offsetsCache.remove(groupTopicAndPartition) + + val commitKey = OffsetManager.offsetCommitKey(groupTopicAndPartition.group, + groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) + + (offsetsPartition, new Message(bytes = null, key = commitKey)) + }.groupBy{ case (partition, tombstone) => partition } + + // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, + // if we crash or leaders move) since the new leaders will get rid of stale offsets during their own purge cycles. + val numRemoved = tombstonesForPartition.flatMap { case(offsetsPartition, tombstones) => + val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + partitionOpt.map { partition => + val appendPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + val messages = tombstones.map(_._2).toSeq + + trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) + + try { + partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) + tombstones.size + } + catch { + case t: Throwable => + error("Failed to mark %d stale offsets for deletion in %s.".format(messages.size, appendPartition), t) + // ignore and continue + 0 + } + } + }.sum + + debug("Removed %d stale offsets in %d milliseconds.".format(numRemoved, SystemTime.milliseconds - startMs)) + } + + def offsetsTopicConfig: Properties = { + val props = new Properties + props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) + props.put(LogConfig.CleanupPolicyProp, "dedupe") + props + } + + def partitionFor(group: String): Int = Utils.abs(group.hashCode) % config.offsetsTopicNumPartitions + + /** + * Fetch the current offset for the given group/topic/partition from the underlying offsets storage. + * + * @param key The requested group-topic-partition + * @return If the key is present, return the offset and metadata; otherwise return None + */ + private def getOffset(key: GroupTopicPartition) = { + val offsetAndMetadata = offsetsCache.get(key) + if (offsetAndMetadata == null) + OffsetMetadataAndError.NoOffset + else + OffsetMetadataAndError(offsetAndMetadata.offset, offsetAndMetadata.metadata, ErrorMapping.NoError) + } + + /** + * Put the (already committed) offset for the given group/topic/partition into the cache. + * + * @param key The group-topic-partition + * @param offsetAndMetadata The offset/metadata to be stored + */ + private def putOffset(key: GroupTopicPartition, offsetAndMetadata: OffsetAndMetadata) { + offsetsCache.put(key, offsetAndMetadata) + } + + def putOffsets(group: String, offsets: Map[TopicAndPartition, OffsetAndMetadata]) { + // this method is called _after_ the offsets have been durably appended to the commit log, so there is no need to + // check for current leadership as we do for the offset fetch + trace("Putting offsets %s for group %s in offsets partition %d.".format(offsets, group, partitionFor(group))) + offsets.foreach { case (topicAndPartition, offsetAndMetadata) => + putOffset(GroupTopicPartition(group, topicAndPartition), offsetAndMetadata) + } + } + + /** + * The most important guarantee that this API provides is that it should never return a stale offset. i.e., it either + * returns the current offset or it begins to sync the cache from the log (and returns an error code). + */ + def getOffsets(group: String, topicPartitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = { + trace("Getting offsets %s for group %s.".format(topicPartitions, group)) + + val offsetsPartition = partitionFor(group) + + /** + * followerTransitionLock protects against fetching from an empty/cleared offset cache (i.e., cleared due to a + * leader->follower transition). i.e., even if leader-is-local is true a follower transition can occur right after + * the check and clear the cache. i.e., we would read from the empty cache and incorrectly return NoOffset. + */ + followerTransitionLock synchronized { + if (leaderIsLocal(offsetsPartition)) { + if (loadingPartitions synchronized loadingPartitions.contains(offsetsPartition)) { + debug("Cannot fetch offsets for group %s due to ongoing offset load.".format(group)) + topicPartitions.map { topicAndPartition => + val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) + (groupTopicPartition.topicPartition, OffsetMetadataAndError.OffsetsLoading) + }.toMap + } else { + if (topicPartitions.size == 0) { + // Return offsets for all partitions owned by this consumer group. (this only applies to consumers that commit offsets to Kafka.) + offsetsCache.filter(_._1.group == group).map { case(groupTopicPartition, offsetAndMetadata) => + (groupTopicPartition.topicPartition, OffsetMetadataAndError(offsetAndMetadata.offset, offsetAndMetadata.metadata, ErrorMapping.NoError)) + }.toMap + } else { + topicPartitions.map { topicAndPartition => + val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) + (groupTopicPartition.topicPartition, getOffset(groupTopicPartition)) + }.toMap + } + } + } else { + debug("Could not fetch offsets for group %s (not offset coordinator).".format(group)) + topicPartitions.map { topicAndPartition => + val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) + (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotOffsetManagerForGroup) + }.toMap + } + } + } + + /** + * Asynchronously read the partition from the offsets topic and populate the cache + */ + def loadOffsetsFromLog(offsetsPartition: Int) { + + val topicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, offsetsPartition) + + loadingPartitions synchronized { + if (loadingPartitions.contains(offsetsPartition)) { + info("Offset load from %s already in progress.".format(topicPartition)) + } else { + loadingPartitions.add(offsetsPartition) + scheduler.schedule(topicPartition.toString, loadOffsets) + } + } + + def loadOffsets() { + info("Loading offsets from " + topicPartition) + + val startMs = SystemTime.milliseconds + try { + replicaManager.logManager.getLog(topicPartition) match { + case Some(log) => + var currOffset = log.logSegments.head.baseOffset + val buffer = ByteBuffer.allocate(config.loadBufferSize) + // loop breaks if leader changes at any time during the load, since getHighWatermark is -1 + while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { + buffer.clear() + val messages = log.read(currOffset, config.loadBufferSize).asInstanceOf[FileMessageSet] + messages.readInto(buffer, 0) + val messageSet = new ByteBufferMessageSet(buffer) + messageSet.foreach { msgAndOffset => + require(msgAndOffset.message.key != null, "Offset entry key should not be null") + val key = OffsetManager.readMessageKey(msgAndOffset.message.key) + if (msgAndOffset.message.payload == null) { + if (offsetsCache.remove(key) != null) + trace("Removed offset for %s due to tombstone entry.".format(key)) + else + trace("Ignoring redundant tombstone for %s.".format(key)) + } else { + val value = OffsetManager.readMessageValue(msgAndOffset.message.payload) + putOffset(key, value) + trace("Loaded offset %s for %s.".format(value, key)) + } + currOffset = msgAndOffset.nextOffset + } + } + + if (!shuttingDown.get()) + info("Finished loading offsets from %s in %d milliseconds." + .format(topicPartition, SystemTime.milliseconds - startMs)) + case None => + warn("No log found for " + topicPartition) + } + } + catch { + case t: Throwable => + error("Error in loading offsets from " + topicPartition, t) + } + finally { + loadingPartitions synchronized loadingPartitions.remove(offsetsPartition) + } + } + } + + private def getHighWatermark(partitionId: Int): Long = { + val partitionOpt = replicaManager.getPartition(OffsetManager.OffsetsTopicName, partitionId) + + val hw = partitionOpt.map { partition => + partition.leaderReplicaIfLocal().map(_.highWatermark).getOrElse(-1L) + }.getOrElse(-1L) + + hw + } + + private def leaderIsLocal(partition: Int) = { getHighWatermark(partition) != -1L } + + /** + * When this broker becomes a follower for an offsets topic partition clear out the cache for groups that belong to + * that partition. + * @param offsetsPartition Groups belonging to this partition of the offsets topic will be deleted from the cache. + */ + def clearOffsetsInPartition(offsetsPartition: Int) { + debug("Deleting offset entries belonging to [%s,%d].".format(OffsetManager.OffsetsTopicName, offsetsPartition)) + + followerTransitionLock synchronized { + offsetsCache.keys.foreach { key => + if (partitionFor(key.group) == offsetsPartition) { + offsetsCache.remove(key) + } + } + } + } + + def shutdown() { + shuttingDown.set(true) + } + +} + +object OffsetManager { + + val OffsetsTopicName = "__consumer_offsets" + + private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) + + private val CURRENT_OFFSET_SCHEMA_VERSION = 0.toShort + + private val OFFSET_COMMIT_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING), + new Field("topic", STRING), + new Field("partition", INT32)) + private val KEY_GROUP_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("group") + private val KEY_TOPIC_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("topic") + private val KEY_PARTITION_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("partition") + + private val OFFSET_COMMIT_VALUE_SCHEMA_V0 = new Schema(new Field("offset", INT64), + new Field("metadata", STRING, "Associated metadata.", ""), + new Field("timestamp", INT64)) + private val VALUE_OFFSET_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") + private val VALUE_METADATA_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") + private val VALUE_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") + + // map of versions to schemas + private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0)) + + private val CURRENT_SCHEMA = schemaFor(CURRENT_OFFSET_SCHEMA_VERSION) + + private def schemaFor(version: Int) = { + val schemaOpt = OFFSET_SCHEMAS.get(version) + schemaOpt match { + case Some(schema) => schema + case _ => throw new KafkaException("Unknown offset schema version " + version) + } + } + + /** + * Generates the key for offset commit message for given (group, topic, partition) + * + * @return key for offset commit message + */ + def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { + val key = new Struct(CURRENT_SCHEMA.keySchema) + key.set(KEY_GROUP_FIELD, group) + key.set(KEY_TOPIC_FIELD, topic) + key.set(KEY_PARTITION_FIELD, partition) + + val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf) + byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + key.writeTo(byteBuffer) + byteBuffer.array() + } + + /** + * Generates the payload for offset commit message from given offset and metadata + * + * @param offsetAndMetadata consumer's current offset and metadata + * @return payload for offset commit message + */ + def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { + val value = new Struct(CURRENT_SCHEMA.valueSchema) + value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) + value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) + value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) + + val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) + byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + value.writeTo(byteBuffer) + byteBuffer.array() + } + + /** + * Decodes the offset messages' key + * + * @param buffer input byte-buffer + * @return an GroupTopicPartition object + */ + def readMessageKey(buffer: ByteBuffer): GroupTopicPartition = { + val version = buffer.getShort() + val keySchema = schemaFor(version).keySchema + val key = keySchema.read(buffer).asInstanceOf[Struct] + + val group = key.get(KEY_GROUP_FIELD).asInstanceOf[String] + val topic = key.get(KEY_TOPIC_FIELD).asInstanceOf[String] + val partition = key.get(KEY_PARTITION_FIELD).asInstanceOf[Int] + + GroupTopicPartition(group, TopicAndPartition(topic, partition)) + } + + /** + * Decodes the offset messages' payload and retrieves offset and metadata from it + * + * @param buffer input byte-buffer + * @return an offset-metadata object from the message + */ + def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { + if(buffer == null) { // tombstone + null + } else { + val version = buffer.getShort() + val valueSchema = schemaFor(version).valueSchema + val value = valueSchema.read(buffer).asInstanceOf[Struct] + + val offset = value.get(VALUE_OFFSET_FIELD).asInstanceOf[Long] + val metadata = value.get(VALUE_METADATA_FIELD).asInstanceOf[String] + val timestamp = value.get(VALUE_TIMESTAMP_FIELD).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, timestamp) + } + } + + // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. + // (specify --formatter "kafka.server.OffsetManager\$OffsetsMessageFormatter" when consuming __consumer_offsets) + class OffsetsMessageFormatter extends MessageFormatter { + def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { + val formattedKey = if (key == null) "NULL" else OffsetManager.readMessageKey(ByteBuffer.wrap(key)).toString + val formattedValue = if (value == null) "NULL" else OffsetManager.readMessageValue(ByteBuffer.wrap(value)).toString + output.write(formattedKey.getBytes) + output.write("::".getBytes) + output.write(formattedValue.getBytes) + output.write("\n".getBytes) + } + } + +} + +case class GroupTopicPartition(group: String, topicPartition: TopicAndPartition) { + + def this(group: String, topic: String, partition: Int) = + this(group, new TopicAndPartition(topic, partition)) + + override def toString = + "[%s,%s,%d]".format(group, topicPartition.topic, topicPartition.partition) + +} + diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index fb759d989147..f16fbe672861 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -205,7 +205,8 @@ class ReplicaManager(val config: KafkaConfig, } } - def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = { + def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest, + offsetManager: OffsetManager): (collection.Map[(String, Int), Short], Short) = { leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => stateChangeLogger.trace("Broker %d received LeaderAndIsr request %s correlation id %d from controller %d epoch %d for partition [%s,%d]" .format(localBrokerId, stateInfo, leaderAndISRRequest.correlationId, @@ -255,8 +256,10 @@ class ReplicaManager(val config: KafkaConfig, .filter{ case (partition, partitionStateInfo) => partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader == config.brokerId} val partitionsToBeFollower = (partitionState -- partitionsTobeLeader.keys) - if (!partitionsTobeLeader.isEmpty) makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap) - if (!partitionsToBeFollower.isEmpty) makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap) + if (!partitionsTobeLeader.isEmpty) + makeLeaders(controllerId, controllerEpoch, partitionsTobeLeader, leaderAndISRRequest.correlationId, responseMap, offsetManager) + if (!partitionsToBeFollower.isEmpty) + makeFollowers(controllerId, controllerEpoch, partitionsToBeFollower, leaderAndISRRequest.leaders, leaderAndISRRequest.correlationId, responseMap, offsetManager) // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions // have been completely populated before starting the checkpointing there by avoiding weird race conditions @@ -283,7 +286,8 @@ class ReplicaManager(val config: KafkaConfig, */ private def makeLeaders(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) = { + correlationId: Int, responseMap: mutable.Map[(String, Int), Short], + offsetManager: OffsetManager) = { partitionState.foreach(state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-leader transition for partition %s") @@ -302,7 +306,7 @@ class ReplicaManager(val config: KafkaConfig, } // Update the partition information to be the leader partitionState.foreach{ case (partition, partitionStateInfo) => - partition.makeLeader(controllerId, partitionStateInfo, correlationId)} + partition.makeLeader(controllerId, partitionStateInfo, correlationId, offsetManager)} // Finally add these partitions to the list of partitions for which the leader is the current broker leaderPartitionsLock synchronized { @@ -344,7 +348,8 @@ class ReplicaManager(val config: KafkaConfig, * the error message will be set on each partition since we do not know which partition caused it */ private def makeFollowers(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - leaders: Set[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) { + leaders: Set[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short], + offsetManager: OffsetManager) { partitionState.foreach { state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-follower transition for partition %s") @@ -367,7 +372,7 @@ class ReplicaManager(val config: KafkaConfig, val newLeaderBrokerId = leaderIsrAndControllerEpoch.leaderAndIsr.leader leaders.find(_.id == newLeaderBrokerId) match { case Some(leaderBroker) => - if (partition.makeFollower(controllerId, partitionStateInfo, correlationId)) + if (partition.makeFollower(controllerId, partitionStateInfo, correlationId, offsetManager)) partitionsToMakeFollower += partition else stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " + diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index 33d7c2c4aacc..88f824fa5afe 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -20,16 +20,22 @@ package kafka.tools import joptsimple._ import org.I0Itec.zkclient.ZkClient -import kafka.utils.{Json, ZkUtils, ZKStringSerializer, Logging} +import kafka.utils._ import kafka.consumer.SimpleConsumer -import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} -import kafka.common.{BrokerNotAvailableException, TopicAndPartition} +import kafka.api.{OffsetFetchResponse, OffsetFetchRequest, OffsetRequest} +import kafka.common.{OffsetMetadataAndError, ErrorMapping, BrokerNotAvailableException, TopicAndPartition} import scala.collection._ +import kafka.client.ClientUtils +import kafka.network.BlockingChannel +import kafka.api.PartitionOffsetRequestInfo +import scala.Some object ConsumerOffsetChecker extends Logging { private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map() + private val offsetMap: mutable.Map[TopicAndPartition, Long] = mutable.Map() + private var topicPidMap: immutable.Map[String, Seq[Int]] = immutable.Map() private def getConsumer(zkClient: ZkClient, bid: Int): Option[SimpleConsumer] = { try { @@ -49,18 +55,17 @@ object ConsumerOffsetChecker extends Logging { } } catch { case t: Throwable => - error("Could not parse broker info", t) + println("Could not parse broker info due to " + t.getCause) None } } private def processPartition(zkClient: ZkClient, group: String, topic: String, pid: Int) { - val offset = ZkUtils.readData(zkClient, "/consumers/%s/offsets/%s/%s". - format(group, topic, pid))._1.toLong - val owner = ZkUtils.readDataMaybeNull(zkClient, "/consumers/%s/owners/%s/%s". - format(group, topic, pid))._1 - + val topicPartition = TopicAndPartition(topic, pid) + val offsetOpt = offsetMap.get(topicPartition) + val groupDirs = new ZKGroupTopicDirs(group, topic) + val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/%s".format(pid))._1 ZkUtils.getLeaderForPartition(zkClient, topic, pid) match { case Some(bid) => val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkClient, bid)) @@ -71,19 +76,18 @@ object ConsumerOffsetChecker extends Logging { OffsetRequest(immutable.Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) val logSize = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head - val lag = logSize - offset - println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offset, logSize, lag, - owner match {case Some(ownerStr) => ownerStr case None => "none"})) + val lagString = offsetOpt.map(o => if (o == -1) "unknown" else (logSize - o).toString) + println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format(group, topic, pid, offsetOpt.getOrElse("unknown"), logSize, lagString.getOrElse("unknown"), + owner match {case Some(ownerStr) => ownerStr case None => "none"})) case None => // ignore } case None => - error("No broker for partition %s - %s".format(topic, pid)) + println("No broker for partition %s - %s".format(topic, pid)) } } private def processTopic(zkClient: ZkClient, group: String, topic: String) { - val pidMap = ZkUtils.getPartitionsForTopics(zkClient, Seq(topic)) - pidMap.get(topic) match { + topicPidMap.get(topic) match { case Some(pids) => pids.sorted.foreach { pid => processPartition(zkClient, group, topic, pid) @@ -105,13 +109,18 @@ object ConsumerOffsetChecker extends Logging { def main(args: Array[String]) { val parser = new OptionParser() - val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string."). - withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]); + val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string."). + withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]) val topicsOpt = parser.accepts("topic", "Comma-separated list of consumer topics (all topics if absent)."). withRequiredArg().ofType(classOf[String]) val groupOpt = parser.accepts("group", "Consumer group."). withRequiredArg().ofType(classOf[String]) + val channelSocketTimeoutMsOpt = parser.accepts("socket.timeout.ms", "Socket timeout to use when querying for offsets."). + withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(6000) + val channelRetryBackoffMsOpt = parser.accepts("retry.backoff.ms", "Retry back-off to use for failed offset queries."). + withRequiredArg().ofType(classOf[java.lang.Integer]).defaultsTo(3000) + parser.accepts("broker-info", "Print broker info") parser.accepts("help", "Print this message.") @@ -122,7 +131,7 @@ object ConsumerOffsetChecker extends Logging { System.exit(0) } - for (opt <- List(groupOpt)) + for (opt <- List(groupOpt, zkConnectOpt)) if (!options.has(opt)) { System.err.println("Missing required argument: %s".format(opt)) parser.printHelpOn(System.err) @@ -130,23 +139,50 @@ object ConsumerOffsetChecker extends Logging { } val zkConnect = options.valueOf(zkConnectOpt) + val group = options.valueOf(groupOpt) - val topics = if (options.has(topicsOpt)) Some(options.valueOf(topicsOpt)) - else None + val groupDirs = new ZKGroupDirs(group) + val channelSocketTimeoutMs = options.valueOf(channelSocketTimeoutMsOpt).intValue() + val channelRetryBackoffMs = options.valueOf(channelRetryBackoffMsOpt).intValue() + + val topics = if (options.has(topicsOpt)) Some(options.valueOf(topicsOpt)) else None var zkClient: ZkClient = null + var channel: BlockingChannel = null try { zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) val topicList = topics match { case Some(x) => x.split(",").view.toList - case None => ZkUtils.getChildren( - zkClient, "/consumers/%s/offsets".format(group)).toList + case None => ZkUtils.getChildren(zkClient, groupDirs.consumerGroupDir + "/owners").toList } - debug("zkConnect = %s; topics = %s; group = %s".format( - zkConnect, topicList.toString(), group)) + topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*) + val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq + + val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) + + debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port)) + channel.send(OffsetFetchRequest(group, topicPartitions)) + val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().buffer) + debug("Received offset fetch response %s.".format(offsetFetchResponse)) + + offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => + if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) { + val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic) + // this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool + // (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage) + val offset = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir + "/%d".format(topicAndPartition.partition))._1.toLong + offsetMap.put(topicAndPartition, offset) + } + else if (offsetAndMetadata.error == ErrorMapping.NoError) + offsetMap.put(topicAndPartition, offsetAndMetadata.offset) + else { + println("Could not fetch offset for %s due to %s.".format(topicAndPartition, ErrorMapping.exceptionFor(offsetAndMetadata.error))) + } + } + channel.disconnect() println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format("Group", "Topic", "Pid", "Offset", "logSize", "Lag", "Owner")) topicList.sorted.foreach { @@ -154,7 +190,7 @@ object ConsumerOffsetChecker extends Logging { } if (options.has("broker-info")) - printBrokerInfo(); + printBrokerInfo() for ((_, consumerOpt) <- consumerMap) consumerOpt match { @@ -162,6 +198,10 @@ object ConsumerOffsetChecker extends Logging { case None => // ignore } } + catch { + case t: Throwable => + println("Exiting due to: %s.".format(t.getMessage)) + } finally { for (consumerOpt <- consumerMap.values) { consumerOpt match { @@ -171,6 +211,9 @@ object ConsumerOffsetChecker extends Logging { } if (zkClient != null) zkClient.close() + + if (channel != null) + channel.disconnect() } } } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 14f44d9305f3..f0ab02a4e6dd 100644 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -62,7 +62,7 @@ object DumpLogSegments { val file = new File(arg) if(file.getName.endsWith(Log.LogFileSuffix)) { println("Dumping " + file) - dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration) + dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize) } else if(file.getName.endsWith(Log.IndexFileSuffix)) { println("Dumping " + file) dumpIndex(file, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize) @@ -117,13 +117,15 @@ object DumpLogSegments { private def dumpLog(file: File, printContents: Boolean, nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]], - isDeepIteration: Boolean) { + isDeepIteration: Boolean, + maxMessageSize: Int) { val startOffset = file.getName().split("\\.")(0).toLong println("Starting offset: " + startOffset) val messageSet = new FileMessageSet(file, false) var validBytes = 0L var lastOffset = -1l - for(shallowMessageAndOffset <- messageSet) { // this only does shallow iteration + val shallowIterator = messageSet.iterator(maxMessageSize) + for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration val itr = getIterator(shallowMessageAndOffset, isDeepIteration) for (messageAndOffset <- itr) { val msg = messageAndOffset.message diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index f0f871c22dea..e4d1a86f673f 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -22,7 +22,6 @@ import kafka.utils.{Utils, CommandLineUtils, Logging} import kafka.producer.{KeyedMessage, ProducerConfig, Producer} import scala.collection.JavaConversions._ import java.util.concurrent.CountDownLatch -import java.nio.ByteBuffer import kafka.consumer._ import kafka.serializer._ import collection.mutable.ListBuffer diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index 5e8c56d96317..a64946108049 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -118,7 +118,10 @@ object ReplicaVerificationTool extends Logging { val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs) val brokerMap = topicsMetadataResponse.extractBrokers(topicsMetadataResponse.topicsMetadata) val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter( - topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic)) true else false + topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false)) + true + else + false ) val topicPartitionReplicaList: Seq[TopicPartitionReplica] = filteredTopicMetadata.flatMap( topicMetadataResponse => diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala index eac9af2e1d45..92c0d1f979fb 100644 --- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala +++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala @@ -26,7 +26,7 @@ object VerifyConsumerRebalance extends Logging { val parser = new OptionParser() val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string."). - withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]); + withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]) val groupOpt = parser.accepts("group", "Consumer group."). withRequiredArg().ofType(classOf[String]) parser.accepts("help", "Print this message.") @@ -78,7 +78,7 @@ object VerifyConsumerRebalance extends Logging { * This means that for each partition registered under /brokers/topics/[topic]/[broker-id], an owner exists * under /consumers/[consumer_group]/owners/[topic]/[broker_id-partition_id] */ - val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group) + val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group, excludeInternalTopics = false) val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, consumersPerTopicMap.keySet.toSeq) partitionsPerTopicMap.foreach { partitionsForTopic => diff --git a/core/src/main/scala/kafka/utils/VerifiableProperties.scala b/core/src/main/scala/kafka/utils/VerifiableProperties.scala index b070bb4df117..d40b03ca844e 100644 --- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala +++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala @@ -20,6 +20,8 @@ package kafka.utils import java.util.Properties import java.util.Collections import scala.collection._ +import kafka.message.{CompressionCodec, NoCompressionCodec} + class VerifiableProperties(val props: Properties) extends Logging { private val referenceSet = mutable.HashSet[String]() @@ -193,6 +195,24 @@ class VerifiableProperties(val props: Properties) extends Logging { } } + /** + * Parse compression codec from a property list in either. Codecs may be specified as integers, or as strings. + * See [[kafka.message.CompressionCodec]] for more details. + * @param name The property name + * @param default Default compression codec + * @return compression codec + */ + def getCompressionCodec(name: String, default: CompressionCodec) = { + val prop = getString(name, NoCompressionCodec.name) + try { + CompressionCodec.getCompressionCodec(prop.toInt) + } + catch { + case nfe: NumberFormatException => + CompressionCodec.getCompressionCodec(prop) + } + } + def verify() { info("Verifying properties") val propNames = { diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index a198628e67fa..16bf7e3e87b5 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -676,12 +676,12 @@ object ZkUtils extends Logging { getChildren(zkClient, dirs.consumerRegistryDir) } - def getConsumersPerTopic(zkClient: ZkClient, group: String) : mutable.Map[String, List[String]] = { + def getConsumersPerTopic(zkClient: ZkClient, group: String, excludeInternalTopics: Boolean) : mutable.Map[String, List[String]] = { val dirs = new ZKGroupDirs(group) val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir) val consumersPerTopicMap = new mutable.HashMap[String, List[String]] for (consumer <- consumers) { - val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient) + val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient, excludeInternalTopics) for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) { for (consumerThreadId <- consumerThreadIdSet) consumersPerTopicMap.get(topic) match { diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala new file mode 100644 index 000000000000..83317f07611b --- /dev/null +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -0,0 +1,291 @@ +package other.kafka + +import org.I0Itec.zkclient.ZkClient +import kafka.api._ +import kafka.utils.{ShutdownableThread, ZKStringSerializer} +import scala.collection._ +import kafka.client.ClientUtils +import joptsimple.OptionParser +import kafka.common.{ErrorMapping, OffsetAndMetadata, TopicAndPartition} +import kafka.network.BlockingChannel +import scala.util.Random +import java.io.IOException +import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} +import java.util.concurrent.TimeUnit +import com.yammer.metrics.core.Gauge +import java.util.concurrent.atomic.AtomicInteger +import java.nio.channels.ClosedByInterruptException + + +object TestOffsetManager { + + val random = new Random + val SocketTimeoutMs = 10000 + + class StatsThread(reportingIntervalMs: Long, commitThreads: Seq[CommitThread], fetchThread: FetchThread) + extends ShutdownableThread("stats-thread") { + + def printStats() { + println("--------------------------------------------------------------------------------") + println("Aggregate stats for commits:") + println("Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d".format( + commitThreads.map(_.numErrors.get).sum, + commitThreads.map(_.timer.max()).max, + commitThreads.map(_.timer.min()).min, + commitThreads.map(_.timer.mean()).sum / commitThreads.size, + commitThreads.map(_.numCommits.get).sum)) + println("--------------------------------------------------------------------------------") + commitThreads.foreach(t => println(t.stats)) + println(fetchThread.stats) + } + + override def doWork() { + printStats() + Thread.sleep(reportingIntervalMs) + } + + } + + class CommitThread(id: Int, partitionCount: Int, commitIntervalMs: Long, zkClient: ZkClient) + extends ShutdownableThread("commit-thread") + with KafkaMetricsGroup { + + private val group = "group-" + id + private val metadata = "Metadata from commit thread " + id + private var offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + private var offset = 0L + val numErrors = new AtomicInteger(0) + val numCommits = new AtomicInteger(0) + val timer = newTimer("commit-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS) + private val commitTimer = new KafkaTimer(timer) + val shutdownLock = new Object + + private def ensureConnected() { + if (!offsetsChannel.isConnected) + offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + } + + override def doWork() { + val commitRequest = OffsetCommitRequest(group, mutable.Map((1 to partitionCount).map(TopicAndPartition("topic-" + id, _) -> OffsetAndMetadata(offset, metadata)):_*)) + try { + ensureConnected() + offsetsChannel.send(commitRequest) + numCommits.getAndIncrement + commitTimer.time { + val response = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer) + if (response.commitStatus.exists(_._2 != ErrorMapping.NoError)) numErrors.getAndIncrement + } + offset += 1 + } + catch { + case e1: ClosedByInterruptException => + offsetsChannel.disconnect() + case e2: IOException => + println("Commit thread %d: Error while committing offsets to %s:%d for group %s due to %s.".format(id, offsetsChannel.host, offsetsChannel.port, group, e2)) + offsetsChannel.disconnect() + } + finally { + Thread.sleep(commitIntervalMs) + } + } + + override def shutdown() { + super.shutdown() + awaitShutdown() + offsetsChannel.disconnect() + println("Commit thread %d ended. Last committed offset: %d.".format(id, offset)) + } + + def stats = { + "Commit thread %d :: Error count: %d; Max:%f; Min: %f; Mean: %f; Commit count: %d" + .format(id, numErrors.get(), timer.max(), timer.min(), timer.mean(), numCommits.get()) + } + } + + class FetchThread(numGroups: Int, fetchIntervalMs: Long, zkClient: ZkClient) + extends ShutdownableThread("fetch-thread") + with KafkaMetricsGroup { + + private val timer = newTimer("fetch-thread", TimeUnit.MILLISECONDS, TimeUnit.SECONDS) + private val fetchTimer = new KafkaTimer(timer) + + private val channels = mutable.Map[Int, BlockingChannel]() + private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + + private val numErrors = new AtomicInteger(0) + + override def doWork() { + val id = random.nextInt().abs % numGroups + val group = "group-" + id + try { + metadataChannel.send(ConsumerMetadataRequest(group)) + val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().buffer).coordinator.map(_.id).getOrElse(-1) + + val channel = if (channels.contains(coordinatorId)) + channels(coordinatorId) + else { + val newChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + channels.put(coordinatorId, newChannel) + newChannel + } + + try { + // send the offset fetch request + val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition("topic-"+id, 1))) + channel.send(fetchRequest) + + fetchTimer.time { + val response = OffsetFetchResponse.readFrom(channel.receive().buffer) + if (response.requestInfo.exists(_._2.error != ErrorMapping.NoError)) { + numErrors.getAndIncrement + } + } + } + catch { + case e1: ClosedByInterruptException => + channel.disconnect() + channels.remove(coordinatorId) + case e2: IOException => + println("Error while fetching offset from %s:%d due to %s.".format(channel.host, channel.port, e2)) + channel.disconnect() + channels.remove(coordinatorId) + } + } + catch { + case e: IOException => + println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port)) + metadataChannel.disconnect() + println("Creating new query channel.") + metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + } + finally { + Thread.sleep(fetchIntervalMs) + } + + } + + override def shutdown() { + super.shutdown() + awaitShutdown() + channels.foreach(_._2.disconnect()) + metadataChannel.disconnect() + } + + def stats = { + "Fetch thread :: Error count: %d; Max:%f; Min: %f; Mean: %f; Fetch count: %d" + .format(numErrors.get(), timer.max(), timer.min(), timer.mean(), timer.count()) + } + } + + def main(args: Array[String]) { + val parser = new OptionParser + val zookeeperOpt = parser.accepts("zookeeper", "The ZooKeeper connection URL.") + .withRequiredArg + .describedAs("ZooKeeper URL") + .ofType(classOf[java.lang.String]) + .defaultsTo("localhost:2181") + + val commitIntervalOpt = parser.accepts("commit-interval-ms", "Offset commit interval.") + .withRequiredArg + .describedAs("interval") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(100) + + val fetchIntervalOpt = parser.accepts("fetch-interval-ms", "Offset fetch interval.") + .withRequiredArg + .describedAs("interval") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1000) + + val numPartitionsOpt = parser.accepts("partition-count", "Number of partitions per commit.") + .withRequiredArg + .describedAs("interval") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) + + val numThreadsOpt = parser.accepts("thread-count", "Number of commit threads.") + .withRequiredArg + .describedAs("threads") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) + + val reportingIntervalOpt = parser.accepts("reporting-interval-ms", "Interval at which stats are reported.") + .withRequiredArg + .describedAs("interval (ms)") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3000) + + val helpOpt = parser.accepts("help", "Print this message.") + + val options = parser.parse(args : _*) + + if (options.has(helpOpt)) { + parser.printHelpOn(System.out) + System.exit(0) + } + + val commitIntervalMs = options.valueOf(commitIntervalOpt).intValue() + val fetchIntervalMs = options.valueOf(fetchIntervalOpt).intValue() + val threadCount = options.valueOf(numThreadsOpt).intValue() + val partitionCount = options.valueOf(numPartitionsOpt).intValue() + val zookeeper = options.valueOf(zookeeperOpt) + val reportingIntervalMs = options.valueOf(reportingIntervalOpt).intValue() + println("Commit thread count: %d; Partition count: %d, Commit interval: %d ms; Fetch interval: %d ms; Reporting interval: %d ms" + .format(threadCount, partitionCount, commitIntervalMs, fetchIntervalMs, reportingIntervalMs)) + + var zkClient: ZkClient = null + var commitThreads: Seq[CommitThread] = Seq() + var fetchThread: FetchThread = null + var statsThread: StatsThread = null + try { + zkClient = new ZkClient(zookeeper, 6000, 2000, ZKStringSerializer) + commitThreads = (0 to (threadCount-1)).map { threadId => + new CommitThread(threadId, partitionCount, commitIntervalMs, zkClient) + } + + fetchThread = new FetchThread(threadCount, fetchIntervalMs, zkClient) + + val statsThread = new StatsThread(reportingIntervalMs, commitThreads, fetchThread) + + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + cleanShutdown() + statsThread.printStats() + } + }) + + commitThreads.foreach(_.start()) + + fetchThread.start() + + statsThread.start() + + commitThreads.foreach(_.join()) + fetchThread.join() + statsThread.join() + } + catch { + case e: Throwable => + println("Error: ", e) + } + finally { + cleanShutdown() + } + + def cleanShutdown() { + commitThreads.foreach(_.shutdown()) + commitThreads.foreach(_.join()) + if (fetchThread != null) { + fetchThread.shutdown() + fetchThread.join() + } + if (statsThread != null) { + statsThread.shutdown() + statsThread.join() + } + zkClient.close() + } + + } +} + diff --git a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala b/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala deleted file mode 100644 index 31534ca303e9..000000000000 --- a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala +++ /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 kafka - -import consumer._ -import utils.Utils -import java.util.concurrent.CountDownLatch - -object TestZKConsumerOffsets { - def main(args: Array[String]): Unit = { - if(args.length < 1) { - println("USAGE: " + TestZKConsumerOffsets.getClass.getName + " consumer.properties topic latest") - System.exit(1) - } - println("Starting consumer...") - val topic = args(1) - val autoOffsetReset = args(2) - val props = Utils.loadProps(args(0)) - props.put("auto.offset.reset", "largest") - - val config = new ConsumerConfig(props) - val consumerConnector: ConsumerConnector = Consumer.create(config) - val topicMessageStreams = consumerConnector.createMessageStreams(Predef.Map(topic -> 1)) - var threadList = List[ConsumerThread]() - for ((topic, streamList) <- topicMessageStreams) - for (stream <- streamList) - threadList ::= new ConsumerThread(stream) - - for (thread <- threadList) - thread.start - - // attach shutdown handler to catch control-c - Runtime.getRuntime().addShutdownHook(new Thread() { - override def run() = { - consumerConnector.shutdown - threadList.foreach(_.shutdown) - println("consumer threads shutted down") - } - }) - } -} - -private class ConsumerThread(stream: KafkaStream[Array[Byte], Array[Byte]]) extends Thread { - val shutdownLatch = new CountDownLatch(1) - - override def run() { - println("Starting consumer thread..") - for (messageAndMetadata <- stream) { - println("consumed: " + new String(messageAndMetadata.message, "UTF-8")) - } - shutdownLatch.countDown - println("thread shutdown !" ) - } - - def shutdown() { - shutdownLatch.await - } -} diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index dbe078c0a3f8..6db76a53173a 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -142,13 +142,6 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val offsetResponse = consumer.getOffsetsBefore(offsetRequest) val errorCode = offsetResponse.partitionErrorAndOffsets(topicAndPartition).error assertTrue("Offset request should fail with UnknownTopicOrPartitionCode", errorCode == ErrorMapping.UnknownTopicOrPartitionCode) - // test if offset fetch requests fail during delete topic - val offsetFetchRequest = new OffsetFetchRequest("test-group", Seq(topicAndPartition)) - val offsetFetchResponse = consumer.fetchOffsets(offsetFetchRequest) - val offsetFetchErrorCode = offsetFetchResponse.requestInfo(topicAndPartition).error - assertTrue("Offset fetch request should fail with UnknownTopicOrPartitionCode", - offsetFetchErrorCode == ErrorMapping.UnknownTopicOrPartitionCode) - // TODO: test if offset commit requests fail during delete topic } // restart follower replica follower.startup() diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index eb274d18c716..537844661495 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -23,12 +23,12 @@ import junit.framework.Assert._ import java.nio.ByteBuffer import kafka.message.{Message, ByteBufferMessageSet} import kafka.cluster.Broker -import collection.mutable._ -import kafka.common.{TopicAndPartition, ErrorMapping, OffsetMetadataAndError} +import kafka.common.{OffsetAndMetadata, TopicAndPartition, ErrorMapping, OffsetMetadataAndError} import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.utils.SystemTime -object SerializationTestUtils{ +object SerializationTestUtils { private val topic1 = "test1" private val topic2 = "test2" private val leader1 = 0 @@ -147,17 +147,15 @@ object SerializationTestUtils{ } def createTestOffsetCommitRequest: OffsetCommitRequest = { - new OffsetCommitRequest("group 1", collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="some metadata"), - TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(offset=100L, metadata=OffsetMetadataAndError.NoMetadata) + new OffsetCommitRequest("group 1", collection.mutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) )) } def createTestOffsetCommitResponse: OffsetCommitResponse = { - new OffsetCommitResponse(collection.immutable.Map( - TopicAndPartition(topic1, 0) -> ErrorMapping.NoError, - TopicAndPartition(topic1, 1) -> ErrorMapping.UnknownTopicOrPartitionCode - )) + new OffsetCommitResponse(collection.immutable.Map(TopicAndPartition(topic1, 0) -> ErrorMapping.NoError, + TopicAndPartition(topic1, 1) -> ErrorMapping.NoError)) } def createTestOffsetFetchRequest: OffsetFetchRequest = { @@ -170,11 +168,18 @@ object SerializationTestUtils{ def createTestOffsetFetchResponse: OffsetFetchResponse = { new OffsetFetchResponse(collection.immutable.Map( TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(42L, "some metadata", ErrorMapping.NoError), - TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetMetadataAndError.NoMetadata, - ErrorMapping.UnknownTopicOrPartitionCode) + TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) )) } + def createConsumerMetadataRequest: ConsumerMetadataRequest = { + ConsumerMetadataRequest("group 1", clientId = "client 1") + } + + def createConsumerMetadataResponse: ConsumerMetadataResponse = { + ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError) + } + } class RequestResponseSerializationTest extends JUnitSuite { @@ -193,114 +198,30 @@ class RequestResponseSerializationTest extends JUnitSuite { private val offsetCommitResponse = SerializationTestUtils.createTestOffsetCommitResponse private val offsetFetchRequest = SerializationTestUtils.createTestOffsetFetchRequest private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse - + private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest + private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse @Test def testSerializationAndDeserialization() { - var buffer: ByteBuffer = ByteBuffer.allocate(leaderAndIsrRequest.sizeInBytes()) - leaderAndIsrRequest.writeTo(buffer) - buffer.rewind() - val deserializedLeaderAndIsrRequest = LeaderAndIsrRequest.readFrom(buffer) - assertEquals("The original and deserialzed leaderAndISRRequest should be the same", leaderAndIsrRequest, - deserializedLeaderAndIsrRequest) - - buffer = ByteBuffer.allocate(leaderAndIsrResponse.sizeInBytes()) - leaderAndIsrResponse.writeTo(buffer) - buffer.rewind() - val deserializedLeaderAndIsrResponse = LeaderAndIsrResponse.readFrom(buffer) - assertEquals("The original and deserialzed leaderAndISRResponse should be the same", leaderAndIsrResponse, - deserializedLeaderAndIsrResponse) - - buffer = ByteBuffer.allocate(stopReplicaRequest.sizeInBytes()) - stopReplicaRequest.writeTo(buffer) - buffer.rewind() - val deserializedStopReplicaRequest = StopReplicaRequest.readFrom(buffer) - assertEquals("The original and deserialzed stopReplicaRequest should be the same", stopReplicaRequest, - deserializedStopReplicaRequest) - - buffer = ByteBuffer.allocate(stopReplicaResponse.sizeInBytes()) - stopReplicaResponse.writeTo(buffer) - buffer.rewind() - val deserializedStopReplicaResponse = StopReplicaResponse.readFrom(buffer) - assertEquals("The original and deserialzed stopReplicaResponse should be the same", stopReplicaResponse, - deserializedStopReplicaResponse) - - buffer = ByteBuffer.allocate(producerRequest.sizeInBytes) - producerRequest.writeTo(buffer) - buffer.rewind() - val deserializedProducerRequest = ProducerRequest.readFrom(buffer) - assertEquals("The original and deserialzed producerRequest should be the same", producerRequest, - deserializedProducerRequest) - - buffer = ByteBuffer.allocate(producerResponse.sizeInBytes) - producerResponse.writeTo(buffer) - buffer.rewind() - val deserializedProducerResponse = ProducerResponse.readFrom(buffer) - assertEquals("The original and deserialzed producerResponse should be the same: [%s], [%s]".format(producerResponse, deserializedProducerResponse), producerResponse, - deserializedProducerResponse) - - buffer = ByteBuffer.allocate(fetchRequest.sizeInBytes) - fetchRequest.writeTo(buffer) - buffer.rewind() - val deserializedFetchRequest = FetchRequest.readFrom(buffer) - assertEquals("The original and deserialzed fetchRequest should be the same", fetchRequest, - deserializedFetchRequest) - - buffer = ByteBuffer.allocate(offsetRequest.sizeInBytes) - offsetRequest.writeTo(buffer) - buffer.rewind() - val deserializedOffsetRequest = OffsetRequest.readFrom(buffer) - assertEquals("The original and deserialzed offsetRequest should be the same", offsetRequest, - deserializedOffsetRequest) - - buffer = ByteBuffer.allocate(offsetResponse.sizeInBytes) - offsetResponse.writeTo(buffer) - buffer.rewind() - val deserializedOffsetResponse = OffsetResponse.readFrom(buffer) - assertEquals("The original and deserialzed offsetResponse should be the same", offsetResponse, - deserializedOffsetResponse) - - buffer = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes()) - topicMetadataRequest.writeTo(buffer) - buffer.rewind() - val deserializedTopicMetadataRequest = TopicMetadataRequest.readFrom(buffer) - assertEquals("The original and deserialzed topicMetadataRequest should be the same", topicMetadataRequest, - deserializedTopicMetadataRequest) - - buffer = ByteBuffer.allocate(topicMetadataResponse.sizeInBytes) - topicMetadataResponse.writeTo(buffer) - buffer.rewind() - val deserializedTopicMetadataResponse = TopicMetadataResponse.readFrom(buffer) - assertEquals("The original and deserialzed topicMetadataResponse should be the same", topicMetadataResponse, - deserializedTopicMetadataResponse) - - buffer = ByteBuffer.allocate(offsetCommitRequest.sizeInBytes) - offsetCommitRequest.writeTo(buffer) - buffer.rewind() - val deserializedOffsetCommitRequest = OffsetCommitRequest.readFrom(buffer) - assertEquals("The original and deserialzed offsetCommitRequest should be the same", offsetCommitRequest, - deserializedOffsetCommitRequest) - - buffer = ByteBuffer.allocate(offsetCommitResponse.sizeInBytes) - offsetCommitResponse.writeTo(buffer) - buffer.rewind() - val deserializedOffsetCommitResponse = OffsetCommitResponse.readFrom(buffer) - assertEquals("The original and deserialzed offsetCommitResponse should be the same", offsetCommitResponse, - deserializedOffsetCommitResponse) - - buffer = ByteBuffer.allocate(offsetFetchRequest.sizeInBytes) - offsetFetchRequest.writeTo(buffer) - buffer.rewind() - val deserializedOffsetFetchRequest = OffsetFetchRequest.readFrom(buffer) - assertEquals("The original and deserialzed offsetFetchRequest should be the same", offsetFetchRequest, - deserializedOffsetFetchRequest) - - buffer = ByteBuffer.allocate(offsetFetchResponse.sizeInBytes) - offsetFetchResponse.writeTo(buffer) - buffer.rewind() - val deserializedOffsetFetchResponse = OffsetFetchResponse.readFrom(buffer) - assertEquals("The original and deserialzed offsetFetchResponse should be the same", offsetFetchResponse, - deserializedOffsetFetchResponse) + val requestsAndResponses = + collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, + stopReplicaRequest, stopReplicaResponse, + producerRequest, producerResponse, + fetchRequest, + offsetRequest, offsetResponse, + topicMetadataRequest, topicMetadataResponse, + offsetCommitRequest, offsetCommitResponse, + offsetFetchRequest, offsetFetchResponse, + consumerMetadataRequest, consumerMetadataResponse) + + requestsAndResponses.foreach { original => + val buffer = ByteBuffer.allocate(original.sizeInBytes) + original.writeTo(buffer) + buffer.rewind() + val deserializer = original.getClass.getDeclaredMethod("readFrom", classOf[ByteBuffer]) + val deserialized = deserializer.invoke(null, buffer) + assertEquals("The original and deserialized request/response should be the same.", original, deserialized) + } } } diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index cf2724bb68d3..d903a6f8e14c 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -21,6 +21,7 @@ package kafka.consumer import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test +import kafka.server.OffsetManager class TopicFilterTest extends JUnitSuite { @@ -29,19 +30,30 @@ class TopicFilterTest extends JUnitSuite { def testWhitelists() { val topicFilter1 = new Whitelist("white1,white2") - assertTrue(topicFilter1.isTopicAllowed("white2")) - assertFalse(topicFilter1.isTopicAllowed("black1")) + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) val topicFilter2 = new Whitelist(".+") - assertTrue(topicFilter2.isTopicAllowed("alltopics")) - + assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true)) + assertFalse(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter2.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) + val topicFilter3 = new Whitelist("white_listed-topic.+") - assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1")) - assertFalse(topicFilter3.isTopicAllowed("black1")) + assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true)) + assertFalse(topicFilter3.isTopicAllowed("black1", excludeInternalTopics = true)) } @Test def testBlacklists() { val topicFilter1 = new Blacklist("black1") + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed("white2", excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) + assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) + + assertFalse(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false)) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 8fe7259c9abd..258dd25a201c 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -115,7 +115,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar assertEquals(expected_1, actual_1) // commit consumed offsets - zkConsumerConnector1.commitOffsets + zkConsumerConnector1.commitOffsets() // create a consumer val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { @@ -194,7 +194,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar assertEquals(expected_1, actual_1) // commit consumed offsets - zkConsumerConnector1.commitOffsets + zkConsumerConnector1.commitOffsets() // create a consumer val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 6a96d80daac5..e63299762dcf 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -23,13 +23,13 @@ import junit.framework.Assert._ import java.util.Properties import kafka.consumer.SimpleConsumer import org.junit.{After, Before, Test} -import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} import kafka.zk.ZooKeeperTestHarness import org.scalatest.junit.JUnit3Suite -import kafka.api.{OffsetCommitRequest, OffsetFetchRequest} +import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest} import kafka.utils.TestUtils._ -import kafka.common.{ErrorMapping, TopicAndPartition, OffsetMetadataAndError} +import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition} import scala.util.Random +import scala.collection._ import kafka.admin.AdminUtils class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -39,6 +39,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { var server: KafkaServer = null var logSize: Int = 100 val brokerPort: Int = 9099 + val group = "test-group" var simpleConsumer: SimpleConsumer = null var time: Time = new MockTime() @@ -51,6 +52,14 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { time = new MockTime() server = TestUtils.createServer(new KafkaConfig(config), time) simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "test-client") + val consumerMetadataRequest = ConsumerMetadataRequest(group) + Stream.continually { + val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest) + consumerMetadataResponse.coordinator.isDefined + }.dropWhile(success => { + if (!success) Thread.sleep(1000) + !success + }) } @After @@ -72,34 +81,34 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError(offset=42L))) + val commitRequest = OffsetCommitRequest("test-group", mutable.Map(topicAndPartition -> OffsetAndMetadata(offset=42L))) val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get) // Fetch it and verify - val fetchRequest = OffsetFetchRequest("test-group", Seq(topicAndPartition)) + val fetchRequest = OffsetFetchRequest(group, Seq(topicAndPartition)) val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(topicAndPartition).get.error) - //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata) + assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata) assertEquals(42L, fetchResponse.requestInfo.get(topicAndPartition).get.offset) // Commit a new offset - val commitRequest1 = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + val commitRequest1 = OffsetCommitRequest(group, mutable.Map(topicAndPartition -> OffsetAndMetadata( offset=100L, metadata="some metadata" ))) val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) - assertEquals(ErrorMapping.NoError, commitResponse1.requestInfo.get(topicAndPartition).get) + assertEquals(ErrorMapping.NoError, commitResponse1.commitStatus.get(topicAndPartition).get) // Fetch it and verify - val fetchRequest1 = OffsetFetchRequest("test-group", Seq(topicAndPartition)) + val fetchRequest1 = OffsetFetchRequest(group, Seq(topicAndPartition)) val fetchResponse1 = simpleConsumer.fetchOffsets(fetchRequest1) - + assertEquals(ErrorMapping.NoError, fetchResponse1.requestInfo.get(topicAndPartition).get.error) - //assertEquals("some metadata", fetchResponse1.requestInfo.get(topicAndPartition).get.metadata) + assertEquals("some metadata", fetchResponse1.requestInfo.get(topicAndPartition).get.metadata) assertEquals(100L, fetchResponse1.requestInfo.get(topicAndPartition).get.offset) } @@ -111,34 +120,19 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val topic3 = "topic-3" val topic4 = "topic-4" - val expectedReplicaAssignment = Map(0 -> List(1)) - // create the topic - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic1, expectedReplicaAssignment) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic2, expectedReplicaAssignment) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic3, expectedReplicaAssignment) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic4, expectedReplicaAssignment) - var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic4, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - - val commitRequest = OffsetCommitRequest("test-group", Map( - TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(offset=42L, metadata="metadata one"), - TopicAndPartition(topic2, 0) -> OffsetMetadataAndError(offset=43L, metadata="metadata two"), - TopicAndPartition(topic3, 0) -> OffsetMetadataAndError(offset=44L, metadata="metadata three"), - TopicAndPartition(topic2, 1) -> OffsetMetadataAndError(offset=45L) + val commitRequest = OffsetCommitRequest("test-group", mutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="metadata one"), + TopicAndPartition(topic2, 0) -> OffsetAndMetadata(offset=43L, metadata="metadata two"), + TopicAndPartition(topic3, 0) -> OffsetAndMetadata(offset=44L, metadata="metadata three"), + TopicAndPartition(topic2, 1) -> OffsetAndMetadata(offset=45L) )) val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic3, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 1)).get) - val fetchRequest = OffsetFetchRequest("test-group", Seq( + val fetchRequest = OffsetFetchRequest(group, Seq( TopicAndPartition(topic1, 0), TopicAndPartition(topic2, 0), TopicAndPartition(topic3, 0), @@ -152,22 +146,22 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) - //assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) - //assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) - //assertEquals("metadata three", fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.metadata) - //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) - //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) - //assertEquals(OffsetMetadataAndError.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) + assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) + assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) + assertEquals("metadata three", fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.metadata) + assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) + assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) + assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset) assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset) assertEquals(44L, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.offset) assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset) - assertEquals(OffsetMetadataAndError.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) - assertEquals(OffsetMetadataAndError.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) + assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) + assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) } @Test @@ -178,36 +172,21 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000) assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + val commitRequest = OffsetCommitRequest("test-group", mutable.Map(topicAndPartition -> OffsetAndMetadata( offset=42L, metadata=random.nextString(server.config.offsetMetadataMaxSize) ))) val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get) - val commitRequest1 = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( + val commitRequest1 = OffsetCommitRequest(group, mutable.Map(topicAndPartition -> OffsetAndMetadata( offset=42L, metadata=random.nextString(server.config.offsetMetadataMaxSize + 1) ))) val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) - assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.requestInfo.get(topicAndPartition).get) - - } + assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.commitStatus.get(topicAndPartition).get) - @Test - def testNullMetadata() { - val topicAndPartition = TopicAndPartition("null-metadata", 0) - val expectedReplicaAssignment = Map(0 -> List(1)) - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topicAndPartition.topic, expectedReplicaAssignment) - var leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicAndPartition.topic, 0, 1000) - assertTrue("Leader should be elected after topic creation", leaderIdOpt.isDefined) - val commitRequest = OffsetCommitRequest("test-group", Map(topicAndPartition -> OffsetMetadataAndError( - offset=42L, - metadata=null - ))) - val commitResponse = simpleConsumer.commitOffsets(commitRequest) - assertEquals(ErrorMapping.NoError, commitResponse.requestInfo.get(topicAndPartition).get) } } diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 1317b4c3c60b..22bb6f2847b8 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -86,12 +86,15 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() EasyMock.replay(replicaManager) + val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) + val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) // start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary) // don't provide replica or leader callbacks since they will not be tested here val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) + val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) + val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo) EasyMock.replay(partitionStateInfo) @@ -159,10 +162,12 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() EasyMock.replay(replicaManager) + val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) + val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, zkClient, configs.head.brokerId, configs.head, controller) + val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) apis.metadataCache.put(TopicAndPartition(topic, partitionId), partitionStateInfo) EasyMock.replay(partitionStateInfo) diff --git a/system_test/migration_tool_testsuite/migration_tool_test.py b/system_test/migration_tool_testsuite/migration_tool_test.py index 2fecd19212b4..2386a5884fcf 100644 --- a/system_test/migration_tool_testsuite/migration_tool_test.py +++ b/system_test/migration_tool_testsuite/migration_tool_test.py @@ -171,7 +171,7 @@ def runTest(self): time.sleep(5) self.log_message("creating topics") - kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv) self.anonLogger.info("sleeping for 5s") time.sleep(5) diff --git a/system_test/mirror_maker/README b/system_test/mirror_maker/README deleted file mode 100644 index da53c1444616..000000000000 --- a/system_test/mirror_maker/README +++ /dev/null @@ -1,22 +0,0 @@ -This test replicates messages from two source kafka clusters into one target -kafka cluster using the mirror-maker tool. At the end, the messages produced -at the source brokers should match that at the target brokers. - -To run this test, do -bin/run-test.sh - -In the event of failure, by default the brokers and zookeepers remain running -to make it easier to debug the issue - hit Ctrl-C to shut them down. You can -change this behavior by setting the action_on_fail flag in the script to "exit" -or "proceed", in which case a snapshot of all the logs and directories is -placed in the test's base directory. - -It is a good idea to run the test in a loop. E.g.: - -:>/tmp/mirrormaker_test.log -for i in {1..10}; do echo "run $i"; ./bin/run-test.sh 2>1 >> /tmp/mirrormaker_test.log; done -tail -F /tmp/mirrormaker_test.log - -grep -ic passed /tmp/mirrormaker_test.log -grep -ic failed /tmp/mirrormaker_test.log - diff --git a/system_test/mirror_maker/bin/expected.out b/system_test/mirror_maker/bin/expected.out deleted file mode 100644 index 0a1bbafcac4a..000000000000 --- a/system_test/mirror_maker/bin/expected.out +++ /dev/null @@ -1,18 +0,0 @@ -start the servers ... -start producing messages ... -wait for consumer to finish consuming ... -[2011-05-17 14:49:11,605] INFO Creating async producer for broker id = 2 at localhost:9091 (kafka.producer.ProducerPool) -[2011-05-17 14:49:11,606] INFO Creating async producer for broker id = 1 at localhost:9092 (kafka.producer.ProducerPool) -[2011-05-17 14:49:11,607] INFO Creating async producer for broker id = 3 at localhost:9090 (kafka.producer.ProducerPool) -thread 0: 400000 messages sent 3514012.1233 nMsg/sec 3.3453 MBs/sec -[2011-05-17 14:49:34,382] INFO Closing all async producers (kafka.producer.ProducerPool) -[2011-05-17 14:49:34,383] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer) -[2011-05-17 14:49:34,384] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer) -[2011-05-17 14:49:34,385] INFO Closed AsyncProducer (kafka.producer.async.AsyncProducer) -Total Num Messages: 400000 bytes: 79859641 in 22.93 secs -Messages/sec: 17444.3960 -MB/sec: 3.3214 -test passed -stopping the servers -bin/../../../bin/zookeeper-server-start.sh: line 9: 22584 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ -bin/../../../bin/zookeeper-server-start.sh: line 9: 22585 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ diff --git a/system_test/mirror_maker/bin/run-test.sh b/system_test/mirror_maker/bin/run-test.sh deleted file mode 100644 index e5e6c08f2c15..000000000000 --- a/system_test/mirror_maker/bin/run-test.sh +++ /dev/null @@ -1,357 +0,0 @@ -#!/bin/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. - -readonly num_messages=10000 -readonly message_size=100 -readonly action_on_fail="proceed" -# readonly action_on_fail="wait" - -readonly test_start_time="$(date +%s)" - -readonly base_dir=$(dirname $0)/.. - -info() { - echo -e "$(date +"%Y-%m-%d %H:%M:%S") $*" -} - -kill_child_processes() { - isTopmost=$1 - curPid=$2 - childPids=$(ps a -o pid= -o ppid= | grep "${curPid}$" | awk '{print $1;}') - for childPid in $childPids - do - kill_child_processes 0 $childPid - done - if [ $isTopmost -eq 0 ]; then - kill -15 $curPid 2> /dev/null - fi -} - -cleanup() { - info "cleaning up" - - pid_zk_source1= - pid_zk_source2= - pid_zk_target= - pid_kafka_source_1_1= - pid_kafka_source_1_2= - pid_kafka_source_2_1= - pid_kafka_source_2_2= - pid_kafka_target_1_1= - pid_kafka_target_1_2= - pid_producer= - pid_mirrormaker_1= - pid_mirrormaker_2= - - rm -rf /tmp/zookeeper* - - rm -rf /tmp/kafka* -} - -begin_timer() { - t_begin=$(date +%s) -} - -end_timer() { - t_end=$(date +%s) -} - -start_zk() { - info "starting zookeepers" - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_1.properties 2>&1 > $base_dir/zookeeper_source-1.log & - pid_zk_source1=$! - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_2.properties 2>&1 > $base_dir/zookeeper_source-2.log & - pid_zk_source2=$! - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_target.properties 2>&1 > $base_dir/zookeeper_target.log & - pid_zk_target=$! -} - -start_source_servers() { - info "starting source cluster" - - JMX_PORT=1111 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_1.properties 2>&1 > $base_dir/kafka_source-1-1.log & - pid_kafka_source_1_1=$! - JMX_PORT=2222 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_2.properties 2>&1 > $base_dir/kafka_source-1-2.log & - pid_kafka_source_1_2=$! - JMX_PORT=3333 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_1.properties 2>&1 > $base_dir/kafka_source-2-1.log & - pid_kafka_source_2_1=$! - JMX_PORT=4444 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_2.properties 2>&1 > $base_dir/kafka_source-2-2.log & - pid_kafka_source_2_2=$! -} - -start_target_servers() { - info "starting mirror cluster" - JMX_PORT=5555 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_1.properties 2>&1 > $base_dir/kafka_target-1-1.log & - pid_kafka_target_1_1=$! - JMX_PORT=6666 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_2.properties 2>&1 > $base_dir/kafka_target-1-2.log & - pid_kafka_target_1_2=$! -} - -shutdown_servers() { - info "stopping mirror-maker" - if [ "x${pid_mirrormaker_1}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_1}; fi - # sleep to avoid rebalancing during shutdown - sleep 2 - if [ "x${pid_mirrormaker_2}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_2}; fi - - info "stopping producer" - if [ "x${pid_producer}" != "x" ]; then kill_child_processes 0 ${pid_producer}; fi - - info "shutting down target servers" - if [ "x${pid_kafka_target_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_1}; fi - if [ "x${pid_kafka_target_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_2}; fi - sleep 2 - - info "shutting down source servers" - if [ "x${pid_kafka_source_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_1}; fi - if [ "x${pid_kafka_source_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_2}; fi - if [ "x${pid_kafka_source_2_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_1}; fi - if [ "x${pid_kafka_source_2_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_2}; fi - - info "shutting down zookeeper servers" - if [ "x${pid_zk_target}" != "x" ]; then kill_child_processes 0 ${pid_zk_target}; fi - if [ "x${pid_zk_source1}" != "x" ]; then kill_child_processes 0 ${pid_zk_source1}; fi - if [ "x${pid_zk_source2}" != "x" ]; then kill_child_processes 0 ${pid_zk_source2}; fi -} - -start_producer() { - topic=$1 - zk=$2 - info "start producing messages for topic $topic to zookeeper $zk ..." - $base_dir/../../bin/kafka-run-class.sh kafka.perf.ProducerPerformance --brokerinfo zk.connect=$zk --topics $topic --messages $num_messages --message-size $message_size --batch-size 200 --vary-message-size --threads 1 --reporting-interval $num_messages --async 2>&1 > $base_dir/producer_performance.log & - pid_producer=$! -} - -# Usage: wait_partition_done ([kafka-server] [topic] [partition-id])+ -wait_partition_done() { - n_tuples=$(($# / 3)) - - i=1 - while (($#)); do - kafka_server[i]=$1 - topic[i]=$2 - partitionid[i]=$3 - prev_offset[i]=0 - info "\twaiting for partition on server ${kafka_server[i]}, topic ${topic[i]}, partition ${partitionid[i]}" - i=$((i+1)) - shift 3 - done - - all_done=0 - - # set -x - while [[ $all_done != 1 ]]; do - sleep 4 - i=$n_tuples - all_done=1 - for ((i=1; i <= $n_tuples; i++)); do - cur_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server ${kafka_server[i]} --topic ${topic[i]} --partition ${partitionid[i]} --time -1 --offsets 1 | tail -1) - if [ "x$cur_size" != "x${prev_offset[i]}" ]; then - all_done=0 - prev_offset[i]=$cur_size - fi - done - done - -} - -cmp_logs() { - topic=$1 - info "comparing source and target logs for topic $topic" - source_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9090 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9091 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part2_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9092 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part3_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9093 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - target_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9094 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - target_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9095 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - if [ "x$source_part0_size" == "x" ]; then source_part0_size=0; fi - if [ "x$source_part1_size" == "x" ]; then source_part1_size=0; fi - if [ "x$source_part2_size" == "x" ]; then source_part2_size=0; fi - if [ "x$source_part3_size" == "x" ]; then source_part3_size=0; fi - if [ "x$target_part0_size" == "x" ]; then target_part0_size=0; fi - if [ "x$target_part1_size" == "x" ]; then target_part1_size=0; fi - expected_size=$(($source_part0_size + $source_part1_size + $source_part2_size + $source_part3_size)) - actual_size=$(($target_part0_size + $target_part1_size)) - if [ "x$expected_size" != "x$actual_size" ] - then - info "source size: $expected_size target size: $actual_size" - return 1 - else - return 0 - fi -} - -take_fail_snapshot() { - snapshot_dir="$base_dir/failed-${snapshot_prefix}-${test_start_time}" - mkdir $snapshot_dir - for dir in /tmp/zookeeper_source{1..2} /tmp/zookeeper_target /tmp/kafka-source-{1..2}-{1..2}-logs /tmp/kafka-target{1..2}-logs; do - if [ -d $dir ]; then - cp -r $dir $snapshot_dir - fi - done -} - -# Usage: process_test_result -# result: last test result -# action_on_fail: (exit|wait|proceed) -# ("wait" is useful if you want to troubleshoot using zookeeper) -process_test_result() { - result=$1 - if [ $1 -eq 0 ]; then - info "test passed" - else - info "test failed" - case "$2" in - "wait") info "waiting: hit Ctrl-c to quit" - wait - ;; - "exit") shutdown_servers - take_fail_snapshot - exit $result - ;; - *) shutdown_servers - take_fail_snapshot - info "proceeding" - ;; - esac - fi -} - -test_whitelists() { - info "### Testing whitelists" - snapshot_prefix="whitelist-test" - - cleanup - start_zk - start_source_servers - start_target_servers - sleep 4 - - info "starting mirror makers" - JMX_PORT=7777 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & - pid_mirrormaker_1=$! - JMX_PORT=8888 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_2.log & - pid_mirrormaker_2=$! - - begin_timer - - start_producer whitetopic01 localhost:2181 - start_producer whitetopic01 localhost:2182 - info "waiting for whitetopic01 producers to finish producing ..." - wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 kafka://localhost:9092 whitetopic01 0 kafka://localhost:9093 whitetopic01 0 - - start_producer whitetopic02 localhost:2181 - start_producer whitetopic03 localhost:2181 - start_producer whitetopic04 localhost:2182 - info "waiting for whitetopic02,whitetopic03,whitetopic04 producers to finish producing ..." - wait_partition_done kafka://localhost:9090 whitetopic02 0 kafka://localhost:9091 whitetopic02 0 kafka://localhost:9090 whitetopic03 0 kafka://localhost:9091 whitetopic03 0 kafka://localhost:9092 whitetopic04 0 kafka://localhost:9093 whitetopic04 0 - - start_producer blacktopic01 localhost:2182 - info "waiting for blacktopic01 producer to finish producing ..." - wait_partition_done kafka://localhost:9092 blacktopic01 0 kafka://localhost:9093 blacktopic01 0 - - info "waiting for consumer to finish consuming ..." - - wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 kafka://localhost:9094 whitetopic02 0 kafka://localhost:9095 whitetopic02 0 kafka://localhost:9094 whitetopic03 0 kafka://localhost:9095 whitetopic03 0 kafka://localhost:9094 whitetopic04 0 kafka://localhost:9095 whitetopic04 0 - - end_timer - info "embedded consumer took $((t_end - t_begin)) seconds" - - sleep 2 - - # if [[ -d /tmp/kafka-target-1-1-logs/blacktopic01 || /tmp/kafka-target-1-2-logs/blacktopic01 ]]; then - # echo "blacktopic01 found on target cluster" - # result=1 - # else - # cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04 - # result=$? - # fi - - cmp_logs blacktopic01 - - cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04 - result=$? - - return $result -} - -test_blacklists() { - info "### Testing blacklists" - snapshot_prefix="blacklist-test" - cleanup - start_zk - start_source_servers - start_target_servers - sleep 4 - - info "starting mirror maker" - $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/blacklisttest.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --blacklist="black.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & - pid_mirrormaker_1=$! - - start_producer blacktopic01 localhost:2181 - start_producer blacktopic02 localhost:2181 - info "waiting for producer to finish producing blacktopic01,blacktopic02 ..." - wait_partition_done kafka://localhost:9090 blacktopic01 0 kafka://localhost:9091 blacktopic01 0 kafka://localhost:9090 blacktopic02 0 kafka://localhost:9091 blacktopic02 0 - - begin_timer - - start_producer whitetopic01 localhost:2181 - info "waiting for producer to finish producing whitetopic01 ..." - wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 - - info "waiting for consumer to finish consuming ..." - wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 - - end_timer - - info "embedded consumer took $((t_end - t_begin)) seconds" - - sleep 2 - - cmp_logs blacktopic01 || cmp_logs blacktopic02 - if [ $? -eq 0 ]; then - return 1 - fi - - cmp_logs whitetopic01 - return $? -} - -# main test begins - -echo "Test-$test_start_time" - -# Ctrl-c trap. Catches INT signal -trap "shutdown_servers; exit 0" INT - -test_whitelists -result=$? - -process_test_result $result $action_on_fail - -shutdown_servers - -sleep 2 - -test_blacklists -result=$? - -process_test_result $result $action_on_fail - -shutdown_servers - -exit $result - diff --git a/system_test/mirror_maker/config/blacklisttest.consumer.properties b/system_test/mirror_maker/config/blacklisttest.consumer.properties deleted file mode 100644 index ff1201582ff1..000000000000 --- a/system_test/mirror_maker/config/blacklisttest.consumer.properties +++ /dev/null @@ -1,28 +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. -# see kafka.consumer.ConsumerConfig for more details - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -#consumer group id -group.id=group1 -shallow.iterator.enable=true - diff --git a/system_test/mirror_maker/config/mirror_producer.properties b/system_test/mirror_maker/config/mirror_producer.properties deleted file mode 100644 index aa8be6504a7a..000000000000 --- a/system_test/mirror_maker/config/mirror_producer.properties +++ /dev/null @@ -1,30 +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. -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2183 -# broker.list=1:localhost:9094,2:localhost:9095 - -# timeout in ms for connecting to zookeeper -# zk.connection.timeout.ms=1000000 - -producer.type=async - -# to avoid dropping events if the queue is full, wait indefinitely -queue.enqueue.timeout.ms=-1 - -num.producers.per.broker=2 - diff --git a/system_test/mirror_maker/config/server_source_1_1.properties b/system_test/mirror_maker/config/server_source_1_1.properties deleted file mode 100644 index 2f070a74e8bd..000000000000 --- a/system_test/mirror_maker/config/server_source_1_1.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=1 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9090 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-1-1-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=10000000 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_source_1_2.properties b/system_test/mirror_maker/config/server_source_1_2.properties deleted file mode 100644 index f9353e8934dc..000000000000 --- a/system_test/mirror_maker/config/server_source_1_2.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=2 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9091 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-1-2-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_source_2_1.properties b/system_test/mirror_maker/config/server_source_2_1.properties deleted file mode 100644 index daa01ad463ec..000000000000 --- a/system_test/mirror_maker/config/server_source_2_1.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=1 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9092 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-2-1-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_source_2_2.properties b/system_test/mirror_maker/config/server_source_2_2.properties deleted file mode 100644 index be6fdfcc11dc..000000000000 --- a/system_test/mirror_maker/config/server_source_2_2.properties +++ /dev/null @@ -1,76 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=2 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9093 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-source-2-2-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - diff --git a/system_test/mirror_maker/config/server_target_1_1.properties b/system_test/mirror_maker/config/server_target_1_1.properties deleted file mode 100644 index d37955a66dcd..000000000000 --- a/system_test/mirror_maker/config/server_target_1_1.properties +++ /dev/null @@ -1,78 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=1 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9094 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-target-1-1-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2183 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - -# topic partition count map -# topic.partition.count.map=topic1:3, topic2:4 diff --git a/system_test/mirror_maker/config/server_target_1_2.properties b/system_test/mirror_maker/config/server_target_1_2.properties deleted file mode 100644 index aa7546cfc8f5..000000000000 --- a/system_test/mirror_maker/config/server_target_1_2.properties +++ /dev/null @@ -1,78 +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. -# see kafka.server.KafkaConfig for additional details and defaults - -# the id of the broker -broker.id=2 - -# hostname of broker. If not set, will pick up from the value returned -# from getLocalHost. If there are multiple interfaces getLocalHost -# may not be what you want. -# host.name= - -# number of logical partitions on this broker -num.partitions=1 - -# the port the socket server runs on -port=9095 - -# the number of processor threads the socket server uses. Defaults to the number of cores on the machine -num.threads=8 - -# the directory in which to store log files -log.dir=/tmp/kafka-target-1-2-logs - -# the send buffer used by the socket server -socket.send.buffer.bytes=1048576 - -# the receive buffer used by the socket server -socket.receive.buffer.bytes=1048576 - -# the maximum size of a log segment -log.segment.bytes=536870912 - -# the interval between running cleanup on the logs -log.cleanup.interval.mins=1 - -# the minimum age of a log file to eligible for deletion -log.retention.hours=168 - -#the number of messages to accept without flushing the log to disk -log.flush.interval.messages=600 - -#set the following properties to use zookeeper - -# enable connecting to zookeeper -enable.zookeeper=true - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2183 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -# time based topic flush intervals in ms -#log.flush.intervals.ms.per.topic=topic:1000 - -# default time based flush interval in ms -log.flush.interval.ms=1000 - -# time based topic flasher time rate in ms -log.flush.scheduler.interval.ms=1000 - -# topic partition count map -# topic.partition.count.map=topic1:3, topic2:4 diff --git a/system_test/mirror_maker/config/whitelisttest_1.consumer.properties b/system_test/mirror_maker/config/whitelisttest_1.consumer.properties deleted file mode 100644 index ff1201582ff1..000000000000 --- a/system_test/mirror_maker/config/whitelisttest_1.consumer.properties +++ /dev/null @@ -1,28 +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. -# see kafka.consumer.ConsumerConfig for more details - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -#consumer group id -group.id=group1 -shallow.iterator.enable=true - diff --git a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties b/system_test/mirror_maker/config/whitelisttest_2.consumer.properties deleted file mode 100644 index f1a902b100ad..000000000000 --- a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties +++ /dev/null @@ -1,28 +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. -# see kafka.consumer.ConsumerConfig for more details - -# zk connection string -# comma separated host:port pairs, each corresponding to a zk -# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 - -# timeout in ms for connecting to zookeeper -zk.connection.timeout.ms=1000000 - -#consumer group id -group.id=group1 -shallow.iterator.enable=true - diff --git a/system_test/mirror_maker/config/zookeeper_target.properties b/system_test/mirror_maker/config/zookeeper_target.properties deleted file mode 100644 index 55a7eb189d64..000000000000 --- a/system_test/mirror_maker/config/zookeeper_target.properties +++ /dev/null @@ -1,18 +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. -# the directory where the snapshot is stored. -dataDir=/tmp/zookeeper_target -# the port at which the clients will connect -clientPort=2183 diff --git a/system_test/mirror_maker_testsuite/mirror_maker_test.py b/system_test/mirror_maker_testsuite/mirror_maker_test.py index fd1808887421..c0117c64cbb7 100644 --- a/system_test/mirror_maker_testsuite/mirror_maker_test.py +++ b/system_test/mirror_maker_testsuite/mirror_maker_test.py @@ -166,7 +166,7 @@ def runTest(self): time.sleep(5) self.log_message("creating topics") - kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv) self.anonLogger.info("sleeping for 5s") time.sleep(5) diff --git a/system_test/offset_management_testsuite/cluster_config.json b/system_test/offset_management_testsuite/cluster_config.json new file mode 100644 index 000000000000..dcca2007de4b --- /dev/null +++ b/system_test/offset_management_testsuite/cluster_config.json @@ -0,0 +1,103 @@ +{ + "cluster_config": [ + { + "entity_id": "0", + "hostname": "localhost", + "role": "zookeeper", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9100" + }, + { + "entity_id": "1", + "hostname": "localhost", + "role": "broker", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9101" + }, + { + "entity_id": "2", + "hostname": "localhost", + "role": "broker", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9102" + }, + { + "entity_id": "3", + "hostname": "localhost", + "role": "broker", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9103" + }, + { + "entity_id": "4", + "hostname": "localhost", + "role": "broker", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9104" + }, + { + "entity_id": "5", + "hostname": "localhost", + "role": "producer_performance", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9105" + }, + { + "entity_id": "6", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9106" + }, + { + "entity_id": "7", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9107" + }, + { + "entity_id": "8", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9108" + }, + { + "entity_id": "9", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9109" + }, + { + "entity_id": "10", + "hostname": "localhost", + "role": "console_consumer", + "cluster_name":"source", + "kafka_home": "default", + "java_home": "default", + "jmx_port": "9110" + } + ] +} diff --git a/system_test/offset_management_testsuite/config/console_consumer.properties b/system_test/offset_management_testsuite/config/console_consumer.properties new file mode 100644 index 000000000000..a2ab8b9c155e --- /dev/null +++ b/system_test/offset_management_testsuite/config/console_consumer.properties @@ -0,0 +1,2 @@ +auto.offset.reset=smallest +auto.commit.interval.ms=1000 diff --git a/system_test/offset_management_testsuite/config/producer_performance.properties b/system_test/offset_management_testsuite/config/producer_performance.properties new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/system_test/offset_management_testsuite/config/server.properties b/system_test/offset_management_testsuite/config/server.properties new file mode 100644 index 000000000000..2b988f86052a --- /dev/null +++ b/system_test/offset_management_testsuite/config/server.properties @@ -0,0 +1,144 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=0 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9091 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=5 + +# Overrides for for the default given by num.partitions on a per-topic basis +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +log.flush.interval.ms=1000 + +# Per-topic overrides for log.flush.interval.ms +#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=102400 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.cleanup.interval.mins=1 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2181 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +controller.message.queue.size=10 +default.replication.factor=1 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + diff --git a/system_test/mirror_maker/config/zookeeper_source_1.properties b/system_test/offset_management_testsuite/config/zookeeper.properties similarity index 83% rename from system_test/mirror_maker/config/zookeeper_source_1.properties rename to system_test/offset_management_testsuite/config/zookeeper.properties index f85179616ebb..5474a72be1a2 100644 --- a/system_test/mirror_maker/config/zookeeper_source_1.properties +++ b/system_test/offset_management_testsuite/config/zookeeper.properties @@ -13,6 +13,11 @@ # See the License for the specific language governing permissions and # limitations under the License. # the directory where the snapshot is stored. -dataDir=/tmp/zookeeper_source-1 +dataDir=/tmp/zookeeper # the port at which the clients will connect clientPort=2181 +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 +syncLimit=5 +initLimit=10 +tickTime=2000 diff --git a/system_test/offset_management_testsuite/offset_management_test.py b/system_test/offset_management_testsuite/offset_management_test.py new file mode 100644 index 000000000000..12b5cd25140e --- /dev/null +++ b/system_test/offset_management_testsuite/offset_management_test.py @@ -0,0 +1,298 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +#!/usr/bin/env python + +# =================================== +# offset_management_test.py +# =================================== + +import os +import signal +import sys +import time +import traceback + +from system_test_env import SystemTestEnv +sys.path.append(SystemTestEnv.SYSTEM_TEST_UTIL_DIR) + +from setup_utils import SetupUtils +from replication_utils import ReplicationUtils +import system_test_utils +from testcase_env import TestcaseEnv + +# product specific: Kafka +import kafka_system_test_utils +import metrics + +class OffsetManagementTest(ReplicationUtils, SetupUtils): + + testModuleAbsPathName = os.path.realpath(__file__) + testSuiteAbsPathName = os.path.abspath(os.path.dirname(testModuleAbsPathName)) + + def __init__(self, systemTestEnv): + + # SystemTestEnv - provides cluster level environment settings + # such as entity_id, hostname, kafka_home, java_home which + # are available in a list of dictionary named + # "clusterEntityConfigDictList" + self.systemTestEnv = systemTestEnv + + super(OffsetManagementTest, self).__init__(self) + + # dict to pass user-defined attributes to logger argument: "extra" + d = {'name_of_class': self.__class__.__name__} + + def signal_handler(self, signal, frame): + self.log_message("Interrupt detected - User pressed Ctrl+c") + + # perform the necessary cleanup here when user presses Ctrl+c and it may be product specific + self.log_message("stopping all entities - please wait ...") + kafka_system_test_utils.stop_all_remote_running_processes(self.systemTestEnv, self.testcaseEnv) + sys.exit(1) + + def runTest(self): + + # ====================================================================== + # get all testcase directories under this testsuite + # ====================================================================== + testCasePathNameList = system_test_utils.get_dir_paths_with_prefix( + self.testSuiteAbsPathName, SystemTestEnv.SYSTEM_TEST_CASE_PREFIX) + testCasePathNameList.sort() + + replicationUtils = ReplicationUtils(self) + + # ============================================================= + # launch each testcase one by one: testcase_1, testcase_2, ... + # ============================================================= + for testCasePathName in testCasePathNameList: + + skipThisTestCase = False + + try: + # ====================================================================== + # A new instance of TestcaseEnv to keep track of this testcase's env vars + # and initialize some env vars as testCasePathName is available now + # ====================================================================== + self.testcaseEnv = TestcaseEnv(self.systemTestEnv, self) + self.testcaseEnv.testSuiteBaseDir = self.testSuiteAbsPathName + self.testcaseEnv.initWithKnownTestCasePathName(testCasePathName) + self.testcaseEnv.testcaseArgumentsDict = self.testcaseEnv.testcaseNonEntityDataDict["testcase_args"] + + # ====================================================================== + # SKIP if this case is IN testcase_to_skip.json or NOT IN testcase_to_run.json + # ====================================================================== + testcaseDirName = self.testcaseEnv.testcaseResultsDict["_test_case_name"] + + if self.systemTestEnv.printTestDescriptionsOnly: + self.testcaseEnv.printTestCaseDescription(testcaseDirName) + continue + elif self.systemTestEnv.isTestCaseToSkip(self.__class__.__name__, testcaseDirName): + self.log_message("Skipping : " + testcaseDirName) + skipThisTestCase = True + continue + else: + self.testcaseEnv.printTestCaseDescription(testcaseDirName) + system_test_utils.setup_remote_hosts_with_testcase_level_cluster_config(self.systemTestEnv, testCasePathName) + + # ============================================================================== # + # ============================================================================== # + # Product Specific Testing Code Starts Here: # + # ============================================================================== # + # ============================================================================== # + + # initialize self.testcaseEnv with user-defined environment variables (product specific) + self.testcaseEnv.userDefinedEnvVarDict["stopBackgroundProducer"] = False + self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"] = False + + # initialize signal handler + signal.signal(signal.SIGINT, self.signal_handler) + + # TestcaseEnv.testcaseConfigsList initialized by reading testcase properties file: + # system_test/_testsuite/testcase_/testcase__properties.json + self.testcaseEnv.testcaseConfigsList = system_test_utils.get_json_list_data( + self.testcaseEnv.testcasePropJsonPathName) + + # clean up data directories specified in zookeeper.properties and kafka_server_.properties + kafka_system_test_utils.cleanup_data_at_remote_hosts(self.systemTestEnv, self.testcaseEnv) + + # create "LOCAL" log directories for metrics, dashboards for each entity under this testcase + # for collecting logs from remote machines + kafka_system_test_utils.generate_testcase_log_dirs(self.systemTestEnv, self.testcaseEnv) + + # TestcaseEnv - initialize producer & consumer config / log file pathnames + kafka_system_test_utils.init_entity_props(self.systemTestEnv, self.testcaseEnv) + + # generate remote hosts log/config dirs if not exist + kafka_system_test_utils.generate_testcase_log_dirs_in_remote_hosts(self.systemTestEnv, self.testcaseEnv) + + # generate properties files for zookeeper, kafka, producer, and consumer: + # 1. copy system_test/_testsuite/config/*.properties to + # system_test/_testsuite/testcase_/config/ + # 2. update all properties files in system_test/_testsuite/testcase_/config + # by overriding the settings specified in: + # system_test/_testsuite/testcase_/testcase__properties.json + kafka_system_test_utils.generate_overriden_props_files(self.testSuiteAbsPathName, + self.testcaseEnv, self.systemTestEnv) + + # ============================================= + # preparing all entities to start the test + # ============================================= + self.log_message("starting zookeepers") + kafka_system_test_utils.start_zookeepers(self.systemTestEnv, self.testcaseEnv) + self.anonLogger.info("sleeping for 2s") + time.sleep(2) + + self.log_message("starting brokers") + kafka_system_test_utils.start_brokers(self.systemTestEnv, self.testcaseEnv) + self.anonLogger.info("sleeping for 5s") + time.sleep(5) + + self.log_message("creating offset topic") + kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 3, 2) + self.anonLogger.info("sleeping for 5s") + time.sleep(5) + + # ============================================= + # starting producer + # ============================================= + self.log_message("starting producer in the background") + kafka_system_test_utils.start_producer_performance(self.systemTestEnv, self.testcaseEnv, False) + msgProducingFreeTimeSec = self.testcaseEnv.testcaseArgumentsDict["message_producing_free_time_sec"] + self.anonLogger.info("sleeping for " + msgProducingFreeTimeSec + " sec to produce some messages") + time.sleep(int(msgProducingFreeTimeSec)) + + kafka_system_test_utils.start_console_consumers(self.systemTestEnv, self.testcaseEnv) + + kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 0) + + # ============================================= + # A while-loop to bounce consumers as specified + # by "num_iterations" in testcase_n_properties.json + # ============================================= + i = 1 + numIterations = int(self.testcaseEnv.testcaseArgumentsDict["num_iteration"]) + bouncedEntityDownTimeSec = 10 + try: + bouncedEntityDownTimeSec = int(self.testcaseEnv.testcaseArgumentsDict["bounced_entity_downtime_sec"]) + except: + pass + + # group1 -> offsets partition 0 // has one consumer; eid: 6 + # group2 -> offsets partition 1 // has four consumers; eid: 7, 8, 9, 10 + + offsets_0_leader_entity = kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 0) + offsets_1_leader_entity = kafka_system_test_utils.get_leader_for(self.systemTestEnv, self.testcaseEnv, "__consumer_offsets", 1) + + while i <= numIterations: + + self.log_message("Iteration " + str(i) + " of " + str(numIterations)) + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, offsets_0_leader_entity, self.testcaseEnv.entityBrokerParentPidDict[offsets_0_leader_entity]) + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, offsets_1_leader_entity, self.testcaseEnv.entityBrokerParentPidDict[offsets_1_leader_entity]) + + # ============================================= + # Bounce consumers if specified in testcase config + # ============================================= + bounceConsumers = self.testcaseEnv.testcaseArgumentsDict["bounce_consumers"] + self.log_message("bounce_consumers flag : " + bounceConsumers) + if (bounceConsumers.lower() == "true"): + + clusterConfigList = self.systemTestEnv.clusterEntityConfigDictList + consumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( clusterConfigList, "role", "console_consumer", "entity_id") + + for stoppedConsumerEntityId in consumerEntityIdList: + consumerPPID = self.testcaseEnv.entityConsoleConsumerParentPidDict[stoppedConsumerEntityId] + self.log_message("stopping consumer: " + consumerPPID) + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, stoppedConsumerEntityId, consumerPPID) + + self.anonLogger.info("sleeping for " + str(bouncedEntityDownTimeSec) + " sec") + time.sleep(bouncedEntityDownTimeSec) + # leaders would have changed during the above bounce. + self.log_message("starting the previously terminated consumers.") + for stoppedConsumerEntityId in consumerEntityIdList: + # starting previously terminated consumer + kafka_system_test_utils.start_console_consumers(self.systemTestEnv, self.testcaseEnv, stoppedConsumerEntityId) + + self.log_message("starting the previously terminated brokers") + kafka_system_test_utils.start_entity_in_background(self.systemTestEnv, self.testcaseEnv, offsets_0_leader_entity) + kafka_system_test_utils.start_entity_in_background(self.systemTestEnv, self.testcaseEnv, offsets_1_leader_entity) + + self.anonLogger.info("sleeping for 15s") + time.sleep(15) + i += 1 + # while loop + + # ============================================= + # tell producer to stop + # ============================================= + self.testcaseEnv.lock.acquire() + self.testcaseEnv.userDefinedEnvVarDict["stopBackgroundProducer"] = True + time.sleep(1) + self.testcaseEnv.lock.release() + time.sleep(1) + + # ============================================= + # wait for producer thread's update of + # "backgroundProducerStopped" to be "True" + # ============================================= + while 1: + self.testcaseEnv.lock.acquire() + self.logger.info("status of backgroundProducerStopped : [" + \ + str(self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]) + "]", extra=self.d) + if self.testcaseEnv.userDefinedEnvVarDict["backgroundProducerStopped"]: + time.sleep(1) + self.logger.info("all producer threads completed", extra=self.d) + break + time.sleep(1) + self.testcaseEnv.lock.release() + time.sleep(2) + + self.anonLogger.info("sleeping for 15s") + time.sleep(15) + + # ============================================= + # this testcase is completed - stop all entities + # ============================================= + self.log_message("stopping all entities") + for entityId, parentPid in self.testcaseEnv.entityBrokerParentPidDict.items(): + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, entityId, parentPid) + + for entityId, parentPid in self.testcaseEnv.entityZkParentPidDict.items(): + kafka_system_test_utils.stop_remote_entity(self.systemTestEnv, entityId, parentPid) + + # make sure all entities are stopped + kafka_system_test_utils.ps_grep_terminate_running_entity(self.systemTestEnv) + + # ============================================= + # collect logs from remote hosts + # ============================================= + kafka_system_test_utils.collect_logs_from_remote_hosts(self.systemTestEnv, self.testcaseEnv) + + # ============================================= + # validate the data matched and checksum + # ============================================= + self.log_message("validating data matched") + kafka_system_test_utils.validate_data_matched_in_multi_topics_from_single_consumer_producer(self.systemTestEnv, self.testcaseEnv, replicationUtils) + + except Exception as e: + self.log_message("Exception while running test {0}".format(e)) + traceback.print_exc() + + finally: + if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly: + self.log_message("stopping all entities - please wait ...") + kafka_system_test_utils.stop_all_remote_running_processes(self.systemTestEnv, self.testcaseEnv) + diff --git a/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json new file mode 100644 index 000000000000..02af3e86357d --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json @@ -0,0 +1,95 @@ +{ + "description": {"01":"To Test : 'Basic offset management test.'", + "02":"Set up a Zk and Kafka cluster.", + "03":"Produce messages to a multiple topics - various partition counts.", + "04":"Start multiple consumer groups to read various subsets of above topics.", + "05":"Bounce consumers.", + "06":"Verify that there are no duplicate messages or lost messages on any consumer group.", + "07":"Producer dimensions : mode:sync, acks:-1, comp:0" + }, + "testcase_args": { + "bounce_leaders": "false", + "bounce_consumers": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50", + "num_topics_for_auto_generated_string":"1" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_1.log", + "config_filename": "kafka_server_1.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_2.log", + "config_filename": "kafka_server_2.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_3.log", + "config_filename": "kafka_server_3.properties" + }, + { + "entity_id": "4", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "topic": "test", + "threads": "3", + "compression-codec": "0", + "message-size": "500", + "message": "1000", + "request-num-acks": "-1", + "sync":"true", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "6", + "topic": "test_0001", + "group.id": "group1", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_6.properties" + } + ] +} diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties new file mode 100644 index 000000000000..41ec6e49272f --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties @@ -0,0 +1,148 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=1 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9091 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_1_logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=5 + +# Overrides for for the default given by num.partitions on a per-topic basis +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +log.flush.interval.ms=1000 + +# Per-topic overrides for log.flush.interval.ms +#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.cleanup.interval.mins=1 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +controller.message.queue.size=10 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-1/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties new file mode 100644 index 000000000000..727e23701d6c --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties @@ -0,0 +1,148 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=2 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9092 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_2_logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=5 + +# Overrides for for the default given by num.partitions on a per-topic basis +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +log.flush.interval.ms=1000 + +# Per-topic overrides for log.flush.interval.ms +#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.cleanup.interval.mins=1 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +controller.message.queue.size=10 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-2/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties new file mode 100644 index 000000000000..e6fbbe1e0532 --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties @@ -0,0 +1,148 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=3 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9093 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_3_logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=5 + +# Overrides for for the default given by num.partitions on a per-topic basis +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +log.flush.interval.ms=1000 + +# Per-topic overrides for log.flush.interval.ms +#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.cleanup.interval.mins=1 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +controller.message.queue.size=10 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-3/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties new file mode 100644 index 000000000000..fee65bce6356 --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties @@ -0,0 +1,148 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# see kafka.server.KafkaConfig for additional details and defaults + +############################# Server Basics ############################# + +# The id of the broker. This must be set to a unique integer for each broker. +broker.id=4 + +# Hostname the broker will advertise to consumers. If not set, kafka will use the value returned +# from InetAddress.getLocalHost(). If there are multiple interfaces getLocalHost +# may not be what you want. +#host.name= + + +############################# Socket Server Settings ############################# + +# The port the socket server listens on +port=9094 + +# The number of threads handling network requests +num.network.threads=2 + +# The number of threads doing disk I/O +num.io.threads=2 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=1048576 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=1048576 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# The directory under which to store log files +log.dir=/tmp/kafka_server_4_logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=5 + +# Overrides for for the default given by num.partitions on a per-topic basis +#topic.partition.count.map=topic1:3, topic2:4 + +############################# Log Flush Policy ############################# + +# The following configurations control the flush of data to disk. This is the most +# important performance knob in kafka. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data is at greater risk of loss in the event of a crash. +# 2. Latency: Data is not made available to consumers until it is flushed (which adds latency). +# 3. Throughput: The flush is generally the most expensive operation. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +log.flush.interval.ms=1000 + +# Per-topic overrides for log.flush.interval.ms +#log.flush.intervals.ms.per.topic=topic1:1000, topic2:3000 + +# The interval (in ms) at which logs are checked to see if they need to be flushed to disk. +log.flush.scheduler.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log as long as the remaining +# segments don't drop below log.retention.bytes. +#log.retention.bytes=1073741824 +log.retention.bytes=-1 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +#log.segment.size=536870912 +log.segment.bytes=10240 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.cleanup.interval.mins=1 + +############################# Zookeeper ############################# + +# Enable connecting to zookeeper +enable.zookeeper=true + +# Zk connection string (see zk docs for details). +# This is a comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002". +# You can also append an optional chroot string to the urls to specify the +# root directory for all kafka znodes. +zookeeper.connect=localhost:2108 + +# Timeout in ms for connecting to zookeeper +zookeeper.connection.timeout.ms=1000000 + +monitoring.period.secs=1 +message.max.bytes=1000000 +queued.max.requests=500 +log.roll.hours=168 +log.index.size.max.bytes=10485760 +log.index.interval.bytes=4096 +auto.create.topics.enable=true +controller.socket.timeout.ms=30000 +controller.message.queue.size=10 +default.replication.factor=3 +replica.lag.time.max.ms=10000 +replica.lag.max.messages=4000 +replica.socket.timeout.ms=30000 +replica.socket.receive.buffer.bytes=65536 +replica.fetch.max.bytes=1048576 +replica.fetch.wait.max.ms=500 +replica.fetch.min.bytes=4096 +num.replica.fetchers=1 + +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=4 + +kafka.csv.metrics.dir=/home/jkoshy/Projects/kafka/system_test/offset_management_testsuite/testcase_7002/logs/broker-4/metrics +kafka.csv.metrics.reporter.enabled=true +kafka.metrics.polling.interval.secs=5 +kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter diff --git a/system_test/mirror_maker/config/zookeeper_source_2.properties b/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties similarity index 79% rename from system_test/mirror_maker/config/zookeeper_source_2.properties rename to system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties index d534d180daf8..97c07b9cd470 100644 --- a/system_test/mirror_maker/config/zookeeper_source_2.properties +++ b/system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties @@ -13,6 +13,12 @@ # See the License for the specific language governing permissions and # limitations under the License. # the directory where the snapshot is stored. -dataDir=/tmp/zookeeper_source-2 +dataDir=/tmp/zookeeper_0 # the port at which the clients will connect -clientPort=2182 +clientPort=2108 +# disable the per-ip limit on the number of connections since this is a non-production config +maxClientCnxns=0 +syncLimit=5 +initLimit=10 +tickTime=2000 +server.1=localhost:2107:2109 diff --git a/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json new file mode 100644 index 000000000000..fdab69b41dbc --- /dev/null +++ b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json @@ -0,0 +1,127 @@ +{ + "description": {"01":"To Test : 'Basic offset management test.'", + "02":"Set up a Zk and Kafka cluster.", + "03":"Produce messages to a multiple topics - various partition counts.", + "04":"Start multiple consumer groups to read various subsets of above topics.", + "05":"Bounce consumers.", + "06":"Verify that there are no duplicate messages or lost messages on any consumer group.", + "07":"Producer dimensions : mode:sync, acks:-1, comp:0" + }, + "testcase_args": { + "bounce_leaders": "false", + "bounce_consumers": "true", + "replica_factor": "3", + "num_partition": "1", + "num_iteration": "1", + "sleep_seconds_between_producer_calls": "1", + "message_producing_free_time_sec": "15", + "num_messages_to_produce_per_producer_call": "50", + "num_topics_for_auto_generated_string":"3" + }, + "entities": [ + { + "entity_id": "0", + "clientPort": "2108", + "dataDir": "/tmp/zookeeper_0", + "log_filename": "zookeeper_0.log", + "config_filename": "zookeeper_0.properties" + }, + { + "entity_id": "1", + "port": "9091", + "broker.id": "1", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_1_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_1.log", + "config_filename": "kafka_server_1.properties" + }, + { + "entity_id": "2", + "port": "9092", + "broker.id": "2", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_2_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_2.log", + "config_filename": "kafka_server_2.properties" + }, + { + "entity_id": "3", + "port": "9093", + "broker.id": "3", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_3_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_3.log", + "config_filename": "kafka_server_3.properties" + }, + { + "entity_id": "4", + "port": "9094", + "broker.id": "4", + "log.segment.bytes": "10240", + "log.dir": "/tmp/kafka_server_4_logs", + "default.replication.factor": "3", + "num.partitions": "5", + "log_filename": "kafka_server_4.log", + "config_filename": "kafka_server_4.properties" + }, + { + "entity_id": "5", + "topic": "test", + "threads": "5", + "compression-codec": "0", + "message-size": "500", + "message": "1000", + "request-num-acks": "-1", + "sync":"true", + "producer-num-retries":"5", + "log_filename": "producer_performance_10.log", + "config_filename": "producer_performance_10.properties" + }, + { + "entity_id": "6", + "topic": "test_0001", + "group.id": "group1", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_6.properties" + }, + { + "entity_id": "7", + "topic": "test_0002", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_7.properties" + }, + { + "entity_id": "8", + "topic": "test_0002", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_8.properties" + }, + { + "entity_id": "9", + "topic": "test_0002", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_9.properties" + }, + { + "entity_id": "10", + "topic": "test_0003", + "group.id": "group2", + "consumer-timeout-ms": "30000", + "log_filename": "console_consumer.log", + "config_filename": "console_consumer_10.properties" + } + ] +} diff --git a/system_test/replication_testsuite/replica_basic_test.py b/system_test/replication_testsuite/replica_basic_test.py index 5d3d93ed0bb5..660006cc253b 100644 --- a/system_test/replication_testsuite/replica_basic_test.py +++ b/system_test/replication_testsuite/replica_basic_test.py @@ -188,7 +188,7 @@ def runTest(self): if autoCreateTopic.lower() == "false": self.log_message("creating topics") - kafka_system_test_utils.create_topic(self.systemTestEnv, self.testcaseEnv) + kafka_system_test_utils.create_topic_for_producer_performance(self.systemTestEnv, self.testcaseEnv) self.anonLogger.info("sleeping for 5s") time.sleep(5) diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index 29ab2ba4a6f2..35f2d1b0464e 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -539,6 +539,17 @@ def start_brokers(systemTestEnv, testcaseEnv): for brokerEntityId in brokerEntityIdList: start_entity_in_background(systemTestEnv, testcaseEnv, brokerEntityId) +def start_console_consumers(systemTestEnv, testcaseEnv, onlyThisEntityId=None): + + if onlyThisEntityId is not None: + start_entity_in_background(systemTestEnv, testcaseEnv, onlyThisEntityId) + else: + clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList + consoleConsumerEntityIdList = system_test_utils.get_data_from_list_of_dicts( + clusterEntityConfigDictList, "role", "console_consumer", "entity_id") + for entityId in consoleConsumerEntityIdList: + start_entity_in_background(systemTestEnv, testcaseEnv, entityId) + def start_mirror_makers(systemTestEnv, testcaseEnv, onlyThisEntityId=None): @@ -751,10 +762,72 @@ def start_entity_in_background(systemTestEnv, testcaseEnv, entityId): logPathName + "/" + logFile + " & echo pid:$! > ", logPathName + "/entity_" + entityId + "_pid'"] + elif role == "console_consumer": + clusterToConsumeFrom = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "cluster_name") + numTopicsForAutoGenString = -1 + try: + numTopicsForAutoGenString = int(testcaseEnv.testcaseArgumentsDict["num_topics_for_auto_generated_string"]) + except: + pass + + topic = "" + if numTopicsForAutoGenString < 0: + topic = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "topic") + else: + topic = generate_topics_string("topic", numTopicsForAutoGenString) + + # update this variable and will be used by data validation functions + testcaseEnv.consumerTopicsString = topic + + # 2. consumer timeout + timeoutMs = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "consumer-timeout-ms") + + # 3. consumer formatter + formatterOption = "" + try: + formatterOption = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "formatter") + except: + pass + + # 4. group + groupOption = "" + try: + groupOption = system_test_utils.get_data_by_lookup_keyval(testcaseConfigsList, "entity_id", entityId, "group.id") + groupOption = "--group " + groupOption + except: + pass + + if len(formatterOption) > 0: + formatterOption = " --formatter " + formatterOption + " " + + # get zookeeper connect string + zkConnectStr = "" + if clusterName == "source": + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] + elif clusterName == "target": + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"] + else: + logger.error("Invalid cluster name : " + clusterName, extra=d) + sys.exit(1) + cmdList = ["ssh " + hostname, + "'JAVA_HOME=" + javaHome, + "JMX_PORT=" + jmxPort, + kafkaHome + "/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer", + "--zookeeper " + zkConnectStr, + "--topic " + topic, + "--consumer-timeout-ms " + timeoutMs, + "--csv-reporter-enabled", + groupOption, + formatterOption, + "--from-beginning", + " >> " + logPathName + "/" + logFile + " & echo pid:$! > ", + logPathName + "/entity_" + entityId + "_pid'"] + cmdStr = " ".join(cmdList) logger.debug("executing command: [" + cmdStr + "]", extra=d) system_test_utils.async_sys_call(cmdStr) + logger.info("sleeping for 5 seconds.", extra=d) time.sleep(5) pidCmdStr = "ssh " + hostname + " 'cat " + logPathName + "/entity_" + entityId + "_pid' 2> /dev/null" @@ -773,6 +846,8 @@ def start_entity_in_background(systemTestEnv, testcaseEnv, entityId): testcaseEnv.entityBrokerParentPidDict[entityId] = tokens[1] elif role == "mirror_maker": testcaseEnv.entityMirrorMakerParentPidDict[entityId] = tokens[1] + elif role == "console_consumer": + testcaseEnv.entityConsoleConsumerParentPidDict[entityId] = tokens[1] def start_console_consumer(systemTestEnv, testcaseEnv): @@ -1117,7 +1192,7 @@ def stop_remote_entity(systemTestEnv, entityId, parentPid, signalType="SIGTERM") hostname = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", entityId, "hostname") pidStack = system_test_utils.get_remote_child_processes(hostname, parentPid) - logger.debug("terminating (" + signalType + ") process id: " + parentPid + " in host: " + hostname, extra=d) + logger.info("terminating (" + signalType + ") process id: " + parentPid + " in host: " + hostname, extra=d) if signalType.lower() == "sigterm": system_test_utils.sigterm_remote_process(hostname, pidStack) @@ -1138,7 +1213,7 @@ def force_stop_remote_entity(systemTestEnv, entityId, parentPid): system_test_utils.sigkill_remote_process(hostname, pidStack) -def create_topic(systemTestEnv, testcaseEnv): +def create_topic_for_producer_performance(systemTestEnv, testcaseEnv): clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance") @@ -1184,6 +1259,40 @@ def create_topic(systemTestEnv, testcaseEnv): logger.debug("executing command: [" + cmdStr + "]", extra=d) subproc = system_test_utils.sys_call_return_subproc(cmdStr) +def create_topic(systemTestEnv, testcaseEnv, topic, replication_factor, num_partitions): + clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList + zkEntityId = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "role", "zookeeper", "entity_id") + kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", zkEntityId, "kafka_home") + javaHome = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "entity_id", zkEntityId, "java_home") + createTopicBin = kafkaHome + "/bin/kafka-topics.sh --create" + zkConnectStr = "" + zkHost = system_test_utils.get_data_by_lookup_keyval(clusterEntityConfigDictList, "role", "zookeeper", "hostname") + if len(testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"]) > 0: + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"] + elif len(testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"]) > 0: + zkConnectStr = testcaseEnv.userDefinedEnvVarDict["targetZkConnectStr"] + else: + raise Exception("Empty zkConnectStr found") + + testcaseBaseDir = testcaseEnv.testCaseBaseDir + + testcaseBaseDir = replace_kafka_home(testcaseBaseDir, kafkaHome) + + logger.debug("creating topic: [" + topic + "] at: [" + zkConnectStr + "]", extra=d) + cmdList = ["ssh " + zkHost, + "'JAVA_HOME=" + javaHome, + createTopicBin, + " --topic " + topic, + " --zookeeper " + zkConnectStr, + " --replication-factor " + str(replication_factor), + " --partitions " + str(num_partitions) + " >> ", + testcaseBaseDir + "/logs/create_source_cluster_topic.log'"] + + cmdStr = " ".join(cmdList) + logger.info("executing command: [" + cmdStr + "]", extra=d) + subproc = system_test_utils.sys_call_return_subproc(cmdStr) + + def get_message_id(logPathName, topic=""): logLines = open(logPathName, "r").readlines() @@ -1221,7 +1330,7 @@ def get_message_checksum(logPathName): def validate_data_matched(systemTestEnv, testcaseEnv, replicationUtils): - logger.debug("#### Inside validate_data_matched", extra=d) + logger.info("#### Inside validate_data_matched", extra=d) validationStatusDict = testcaseEnv.validationStatusDict clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList @@ -1245,11 +1354,11 @@ def validate_data_matched(systemTestEnv, testcaseEnv, replicationUtils): consumerTopic = system_test_utils.get_data_by_lookup_keyval(testcaseEnv.testcaseConfigsList, "entity_id", consumerEntityId, "topic") if consumerTopic in topic: matchingConsumerEntityId = consumerEntityId - logger.debug("matching consumer entity id found", extra=d) + logger.info("matching consumer entity id found", extra=d) break if matchingConsumerEntityId is None: - logger.debug("matching consumer entity id NOT found", extra=d) + logger.info("matching consumer entity id NOT found", extra=d) break msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( \ @@ -1337,7 +1446,7 @@ def cleanup_data_at_remote_hosts(systemTestEnv, testcaseEnv): logger.info("cleaning up test case dir: [" + testCaseBaseDir + "]", extra=d) if "system_test" not in testCaseBaseDir: - logger.warn("possible destructive command [" + cmdStr + "]", extra=d) + # logger.warn("possible destructive command [" + cmdStr + "]", extra=d) logger.warn("check config file: system_test/cluster_config.properties", extra=d) logger.warn("aborting test...", extra=d) sys.exit(1) @@ -1539,6 +1648,9 @@ def stop_all_remote_running_processes(systemTestEnv, testcaseEnv): for entityId, mirrorMakerParentPid in testcaseEnv.entityMirrorMakerParentPidDict.items(): stop_remote_entity(systemTestEnv, entityId, mirrorMakerParentPid) + for entityId, consumerParentPid in testcaseEnv.entityConsoleConsumerParentPidDict.items(): + stop_remote_entity(systemTestEnv, entityId, consumerParentPid) + for entityId, brokerParentPid in testcaseEnv.entityBrokerParentPidDict.items(): stop_remote_entity(systemTestEnv, entityId, brokerParentPid) @@ -2119,7 +2231,6 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe clusterEntityConfigDictList = systemTestEnv.clusterEntityConfigDictList prodPerfCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "producer_performance") - consumerCfgList = system_test_utils.get_dict_from_list_of_dicts(clusterEntityConfigDictList, "role", "console_consumer") for prodPerfCfg in prodPerfCfgList: producerEntityId = prodPerfCfg["entity_id"] @@ -2147,7 +2258,7 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe topicList = topicStr.split(',') for topic in topicList: consumerDuplicateCount = 0 - msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( + msgIdMissingInConsumerLogPathName = get_testcase_config_log_dir_pathname( testcaseEnv, "console_consumer", matchingConsumerEntityId, "default") \ + "/msg_id_missing_in_consumer_" + topic + ".log" producerMsgIdList = get_message_id(producerLogPathName, topic) @@ -2163,8 +2274,11 @@ def validate_data_matched_in_multi_topics_from_single_consumer_producer(systemTe outfile.write(id + "\n") outfile.close() + logger.info("Producer entity id " + producerEntityId, extra=d) + logger.info("Consumer entity id " + matchingConsumerEntityId, extra=d) logger.info("no. of unique messages on topic [" + topic + "] sent from publisher : " + str(len(producerMsgIdSet)), extra=d) logger.info("no. of unique messages on topic [" + topic + "] received by consumer : " + str(len(consumerMsgIdSet)), extra=d) + logger.info("no. of duplicate messages on topic [" + topic + "] received by consumer: " + str(consumerDuplicateCount), extra=d) validationStatusDict["Unique messages from producer on [" + topic + "]"] = str(len(producerMsgIdSet)) validationStatusDict["Unique messages from consumer on [" + topic + "]"] = str(len(consumerMsgIdSet)) @@ -2263,6 +2377,43 @@ def validate_index_log(systemTestEnv, testcaseEnv, clusterName="source"): else: validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "FAILED" +def get_leader_for(systemTestEnv, testcaseEnv, topic, partition): + logger.info("Querying Zookeeper for leader info for topic " + topic, extra=d) + clusterConfigsList = systemTestEnv.clusterEntityConfigDictList + tcConfigsList = testcaseEnv.testcaseConfigsList + + zkDictList = system_test_utils.get_dict_from_list_of_dicts(clusterConfigsList, "role", "zookeeper") + firstZkDict = zkDictList[0] + hostname = firstZkDict["hostname"] + zkEntityId = firstZkDict["entity_id"] + clientPort = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "entity_id", zkEntityId, "clientPort") + kafkaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "kafka_home") + javaHome = system_test_utils.get_data_by_lookup_keyval(clusterConfigsList, "entity_id", zkEntityId, "java_home") + kafkaRunClassBin = kafkaHome + "/bin/kafka-run-class.sh" + + zkQueryStr = "get /brokers/topics/" + topic + "/partitions/" + str(partition) + "/state" + brokerid = '' + leaderEntityId = '' + + cmdStrList = ["ssh " + hostname, + "\"JAVA_HOME=" + javaHome, + kafkaRunClassBin + " org.apache.zookeeper.ZooKeeperMain", + "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"], + zkQueryStr + " 2> /dev/null | tail -1\""] + cmdStr = " ".join(cmdStrList) + logger.info("executing command [" + cmdStr + "]", extra=d) + subproc = system_test_utils.sys_call_return_subproc(cmdStr) + for line in subproc.stdout.readlines(): + if "\"leader\"" in line: + line = line.rstrip('\n') + json_data = json.loads(line) + for key,val in json_data.items(): + if key == 'leader': + brokerid = str(val) + leaderEntityId = system_test_utils.get_data_by_lookup_keyval(tcConfigsList, "broker.id", brokerid, "entity_id") + break + return leaderEntityId + def get_leader_attributes(systemTestEnv, testcaseEnv): logger.info("Querying Zookeeper for leader info ...", extra=d) @@ -2294,11 +2445,10 @@ def get_leader_attributes(systemTestEnv, testcaseEnv): "-server " + testcaseEnv.userDefinedEnvVarDict["sourceZkConnectStr"], zkQueryStr + " 2> /dev/null | tail -1\""] cmdStr = " ".join(cmdStrList) - logger.debug("executing command [" + cmdStr + "]", extra=d) + logger.info("executing command [" + cmdStr + "]", extra=d) subproc = system_test_utils.sys_call_return_subproc(cmdStr) for line in subproc.stdout.readlines(): - logger.debug("zk returned : " + line, extra=d) if "\"leader\"" in line: line = line.rstrip('\n') json_data = json.loads(line) diff --git a/system_test/utils/testcase_env.py b/system_test/utils/testcase_env.py index bee87166510b..b3c29105c043 100644 --- a/system_test/utils/testcase_env.py +++ b/system_test/utils/testcase_env.py @@ -51,6 +51,12 @@ class TestcaseEnv(): # { 0: 12345, 1: 12389, ... } entityMirrorMakerParentPidDict = {} + # dictionary of entity_id to ppid for console-consumer entities + # key: entity_id + # val: ppid of console consumer associated to that entity_id + # { 0: 12345, 1: 12389, ... } + entityConsoleConsumerParentPidDict = {} + # dictionary of entity_id to ppid for migration tool entities # key: entity_id # val: ppid of broker associated to that entity_id