From c00cebd93262ab9e0f1a9ea164902ab7ef0c800a Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 15:07:17 -0700 Subject: [PATCH 01/23] move group metadata into offset manager --- .../main/scala/kafka/admin/TopicCommand.scala | 2 +- core/src/main/scala/kafka/common/Topic.scala | 2 +- .../kafka/coordinator/GroupCoordinator.scala | 36 +- .../main/scala/kafka/server/KafkaApis.scala | 10 +- .../scala/kafka/server/OffsetManager.scala | 627 ------------------ core/src/main/scala/kafka/server/package.html | 19 - .../kafka/api/BaseConsumerTest.scala | 2 +- .../kafka/api/IntegrationTestHarness.scala | 2 +- .../unit/kafka/admin/TopicCommandTest.scala | 6 +- .../unit/kafka/consumer/TopicFilterTest.scala | 9 +- 10 files changed, 35 insertions(+), 680 deletions(-) delete mode 100755 core/src/main/scala/kafka/server/OffsetManager.scala delete mode 100644 core/src/main/scala/kafka/server/package.html diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index ed54aee39c66b..8686768978f56 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -129,7 +129,7 @@ object TopicCommand extends Logging { } if(opts.options.has(opts.partitionsOpt)) { - if (topic == GroupCoordinator.OffsetsTopicName) { + if (topic == GroupCoordinator.GroupMetadataTopicName) { 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 " + diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index ca41eba77a4bd..982955e9ae977 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -26,7 +26,7 @@ object Topic { private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") - val InternalTopics = Set(GroupCoordinator.OffsetsTopicName) + val InternalTopics = Set(GroupCoordinator.GroupMetadataTopicName) def validate(topic: String) { if (topic.length <= 0) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 0ef542d485d8a..8fb69efb2f992 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -47,8 +47,8 @@ case class JoinGroupResult(members: Map[String, Array[Byte]], */ class GroupCoordinator(val brokerId: Int, val groupConfig: GroupManagerConfig, - val offsetConfig: OffsetManagerConfig, - private val offsetManager: OffsetManager) extends Logging { + val offsetConfig: OffsetConfig, + private val groupManager: GroupMetadataManager) extends Logging { type JoinCallback = JoinGroupResult => Unit type SyncCallback = (Array[Byte], Short) => Unit @@ -62,11 +62,11 @@ class GroupCoordinator(val brokerId: Int, def this(brokerId: Int, groupConfig: GroupManagerConfig, - offsetConfig: OffsetManagerConfig, + offsetConfig: OffsetConfig, replicaManager: ReplicaManager, zkUtils: ZkUtils, scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig, - new OffsetManager(offsetConfig, replicaManager, zkUtils, scheduler)) + new GroupMetadataManager(offsetConfig, replicaManager, zkUtils, scheduler)) def offsetsTopicConfigs: Properties = { val props = new Properties @@ -100,7 +100,7 @@ class GroupCoordinator(val brokerId: Int, def shutdown() { info("Shutting down.") isActive.set(false) - offsetManager.shutdown() + groupManager.shutdown() coordinatorMetadata.shutdown() heartbeatPurgatory.shutdown() joinPurgatory.shutdown() @@ -365,7 +365,7 @@ class GroupCoordinator(val brokerId: Int, if (group == null) { if (generationId < 0) // the group is not relying on Kafka for partition management, so allow the commit - offsetManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) + groupManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) else // the group has failed over to this coordinator (which will be handled in KAFKA-2017), // or this is a request coming from an older generation. either way, reject the commit @@ -381,7 +381,7 @@ class GroupCoordinator(val brokerId: Int, } else if (generationId != group.generationId) { responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code)) } else { - offsetManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) + groupManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) } } } @@ -397,18 +397,18 @@ class GroupCoordinator(val brokerId: Int, } else { // return offsets blindly regardless the current group state since the group may be using // Kafka commit storage without automatic group management - offsetManager.getOffsets(groupId, partitions) + groupManager.getOffsets(groupId, partitions) } } def handleGroupImmigration(offsetTopicPartitionId: Int) = { // TODO we may need to add more logic in KAFKA-2017 - offsetManager.loadOffsetsFromLog(offsetTopicPartitionId) + groupManager.loadGroupsForPartition(offsetTopicPartitionId) } def handleGroupEmigration(offsetTopicPartitionId: Int) = { // TODO we may need to add more logic in KAFKA-2017 - offsetManager.removeOffsetsFromCacheForPartition(offsetTopicPartitionId) + groupManager.removeGroupsForPartition(offsetTopicPartitionId) } private def validGroupId(groupId: String): Boolean = { @@ -585,27 +585,29 @@ class GroupCoordinator(val brokerId: Int, // TODO: add metrics for complete heartbeats } - def partitionFor(group: String): Int = offsetManager.partitionFor(group) + def partitionFor(group: String): Int = groupManager.partitionFor(group) private def shouldKeepMemberAlive(member: MemberMetadata, heartbeatDeadline: Long) = member.awaitingJoinCallback != null || member.awaitingSyncCallback != null || member.latestHeartbeat + member.sessionTimeoutMs > heartbeatDeadline - private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId)) + private def isCoordinatorForGroup(groupId: String) = groupManager.leaderIsLocal(groupManager.partitionFor(groupId)) } object GroupCoordinator { val NoProtocol = "" val NoLeader = "" - val OffsetsTopicName = "__consumer_offsets" + + // TODO: we store both group metadata and offset data here despite the topic name being offsets only + val GroupMetadataTopicName = "__consumer_offsets" def create(config: KafkaConfig, zkUtils: ZkUtils, replicaManager: ReplicaManager, kafkaScheduler: KafkaScheduler): GroupCoordinator = { - val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize, loadBufferSize = config.offsetsLoadBufferSize, offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, @@ -621,8 +623,8 @@ object GroupCoordinator { def create(config: KafkaConfig, zkUtils: ZkUtils, - offsetManager: OffsetManager): GroupCoordinator = { - val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + groupManager: GroupMetadataManager): GroupCoordinator = { + val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize, loadBufferSize = config.offsetsLoadBufferSize, offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, @@ -633,6 +635,6 @@ object GroupCoordinator { val groupConfig = GroupManagerConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs) - new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager) + new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2ef97304d531f..a2aae2985bfef 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -116,12 +116,12 @@ class KafkaApis(val requestChannel: RequestChannel, // for each new leader or follower, call coordinator to handle // consumer group migration result.updatedLeaders.foreach { case partition => - if (partition.topic == GroupCoordinator.OffsetsTopicName) + if (partition.topic == GroupCoordinator.GroupMetadataTopicName) coordinator.handleGroupImmigration(partition.partitionId) } result.updatedFollowers.foreach { case partition => partition.leaderReplicaIdOpt.foreach { leaderReplica => - if (partition.topic == GroupCoordinator.OffsetsTopicName && + if (partition.topic == GroupCoordinator.GroupMetadataTopicName && leaderReplica == brokerId) coordinator.handleGroupEmigration(partition.partitionId) } @@ -528,9 +528,9 @@ class KafkaApis(val requestChannel: RequestChannel, if (topics.size > 0 && topicResponses.size != topics.size) { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => - if (topic == GroupCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) { + if (topic == GroupCoordinator.GroupMetadataTopicName || config.autoCreateTopicsEnable) { try { - if (topic == GroupCoordinator.OffsetsTopicName) { + if (topic == GroupCoordinator.GroupMetadataTopicName) { val aliveBrokers = metadataCache.getAliveBrokers val offsetsTopicReplicationFactor = if (aliveBrokers.length > 0) @@ -671,7 +671,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = coordinator.partitionFor(groupMetadataRequest.group) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(GroupCoordinator.OffsetsTopicName), request.securityProtocol).head + val offsetsTopicMetadata = getTopicMetadata(Set(GroupCoordinator.GroupMetadataTopicName), request.securityProtocol).head val errorResponse = GroupMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, groupMetadataRequest.correlationId) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala deleted file mode 100755 index 967dc6fa955cd..0000000000000 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ /dev/null @@ -1,627 +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.server - -import org.apache.kafka.common.protocol.Errors -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 org.apache.kafka.common.utils.Utils - -import kafka.utils._ -import kafka.common._ -import kafka.log.FileMessageSet -import kafka.message._ -import kafka.metrics.KafkaMetricsGroup -import kafka.common.TopicAndPartition -import kafka.tools.MessageFormatter -import kafka.api.ProducerResponseStatus -import kafka.coordinator.GroupCoordinator - -import scala.Some -import scala.collection._ -import java.io.PrintStream -import java.util.concurrent.atomic.AtomicBoolean -import java.nio.ByteBuffer -import java.util.concurrent.TimeUnit - -import com.yammer.metrics.core.Gauge -import org.I0Itec.zkclient.ZkClient - -/** - * 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 expired 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 = OffsetManagerConfig.DefaultOffsetRetentionMs, - 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 DefaultOffsetRetentionMs = 24*60*60*1000L - val DefaultOffsetsRetentionCheckIntervalMs = 600000L - val DefaultOffsetsTopicNumPartitions = 50 - val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 - val DefaultOffsetsTopicReplicationFactor = 3.toShort - val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec - val DefaultOffsetCommitTimeoutMs = 5000 - val DefaultOffsetCommitRequiredAcks = (-1).toShort -} - -class OffsetManager(val config: OffsetManagerConfig, - replicaManager: ReplicaManager, - zkUtils: ZkUtils, - 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 cleanupOrLoadMutex = new Object - private val shuttingDown = new AtomicBoolean(false) - private val offsetsTopicPartitionCount = getOffsetsTopicPartitionCount - - this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " - - scheduler.schedule(name = "delete-expired-consumer-offsets", - fun = deleteExpiredOffsets, - 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 deleteExpiredOffsets() { - debug("Collecting expired offsets.") - val startMs = SystemTime.milliseconds - - val numExpiredOffsetsRemoved = cleanupOrLoadMutex synchronized { - val expiredOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => - offsetAndMetadata.expireTimestamp < startMs - } - - debug("Found %d expired offsets.".format(expiredOffsets.size)) - - // delete the expired offsets from the table and generate tombstone messages to remove them from the log - val tombstonesForPartition = expiredOffsets.map { case (groupTopicAndPartition, offsetAndMetadata) => - val offsetsPartition = partitionFor(groupTopicAndPartition.group) - trace("Removing expired 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 expired offsets during their own purge cycles. - tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) => - val partitionOpt = replicaManager.getPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition) - partitionOpt.map { partition => - val appendPartition = TopicAndPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition) - val messages = tombstones.map(_._2).toSeq - - trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) - - try { - // do not need to require acks since even if the tombsone is lost, - // it will be appended again in the next purge cycle - partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages: _*)) - tombstones.size - } - catch { - case t: Throwable => - error("Failed to mark %d expired offsets for deletion in %s.".format(messages.size, appendPartition), t) - // ignore and continue - 0 - } - } - }.sum - } - - info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, SystemTime.milliseconds - startMs)) - } - - - def partitionFor(group: String): Int = Utils.abs(group.hashCode) % offsetsTopicPartitionCount - - /** - * 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) - } - - /* - * Check if the offset metadata length is valid - */ - def validateOffsetMetadataLength(metadata: String) : Boolean = { - metadata == null || metadata.length() <= config.maxMetadataSize - } - - /** - * Store offsets by appending it to the replicated log and then inserting to cache - */ - def storeOffsets(groupId: String, - consumerId: String, - generationId: Int, - offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], - responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { - // first filter out partitions with offset metadata size exceeding limit - val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - validateOffsetMetadataLength(offsetAndMetadata.metadata) - } - - // construct the message set to append - val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - new Message( - key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) - ) - }.toSeq - - val offsetTopicPartition = TopicAndPartition(GroupCoordinator.OffsetsTopicName, partitionFor(groupId)) - - val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> - new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) - - // set the callback function to insert offsets into cache after log append completed - def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { - // the append response should only contain the topics partition - if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) - throw new IllegalStateException("Append status %s should only have one partition %s" - .format(responseStatus, offsetTopicPartition)) - - // construct the commit response status and insert - // the offset and metadata to cache if the append status has no error - val status = responseStatus(offsetTopicPartition) - - val responseCode = - if (status.error == ErrorMapping.NoError) { - filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(groupId, topicAndPartition), offsetAndMetadata) - } - ErrorMapping.NoError - } else { - debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" - .format(filteredOffsetMetadata, groupId, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) - - // transform the log append error code to the corresponding the commit status error code - if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) - ErrorMapping.ConsumerCoordinatorNotAvailableCode - else if (status.error == ErrorMapping.NotLeaderForPartitionCode) - ErrorMapping.NotCoordinatorForConsumerCode - else if (status.error == ErrorMapping.MessageSizeTooLargeCode - || status.error == ErrorMapping.MessageSetSizeTooLargeCode - || status.error == ErrorMapping.InvalidFetchSizeCode) - Errors.INVALID_COMMIT_OFFSET_SIZE.code - else - status.error - } - - - // compute the final error codes for the commit response - val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) - (topicAndPartition, responseCode) - else - (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) - } - - // finally trigger the callback logic passed from the API layer - responseCallback(commitStatus) - } - - // call replica manager to append the offset messages - replicaManager.appendMessages( - config.offsetCommitTimeoutMs.toLong, - config.offsetCommitRequiredAcks, - true, // allow appending to internal offset topic - offsetsAndMetadataMessageSet, - putCacheCallback) - } - - /** - * 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.NotCoordinatorForGroup) - }.toMap - } - } - } - - /** - * Asynchronously read the partition from the offsets topic and populate the cache - */ - def loadOffsetsFromLog(offsetsPartition: Int) { - - val topicPartition = TopicAndPartition(GroupCoordinator.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 - cleanupOrLoadMutex synchronized { - while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { - buffer.clear() - val messages = log.read(currOffset, config.loadBufferSize).messageSet.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 { - // special handling for version 0: - // set the expiration time stamp as commit time stamp + server default retention time - val value = OffsetManager.readMessageValue(msgAndOffset.message.payload) - putOffset(key, value.copy ( - expireTimestamp = { - if (value.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - value.commitTimestamp + config.offsetsRetentionMs - else - value.expireTimestamp - } - )) - 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(GroupCoordinator.OffsetsTopicName, partitionId) - - val hw = partitionOpt.map { partition => - partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) - }.getOrElse(-1L) - - hw - } - - 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 removeOffsetsFromCacheForPartition(offsetsPartition: Int) { - var numRemoved = 0 - followerTransitionLock synchronized { - offsetsCache.keys.foreach { key => - if (partitionFor(key.group) == offsetsPartition) { - offsetsCache.remove(key) - numRemoved += 1 - } - } - } - - if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." - .format(numRemoved, TopicAndPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition))) - } - - def shutdown() { - shuttingDown.set(true) - } - - /** - * Gets the partition count of the offsets topic from ZooKeeper. - * If the topic does not exist, the configured partition count is returned. - */ - private def getOffsetsTopicPartitionCount = { - val topic = GroupCoordinator.OffsetsTopicName - val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic)) - if (topicData(topic).nonEmpty) - topicData(topic).size - else - config.offsetsTopicNumPartitions - } -} - -object OffsetManager { - - private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) - - private val CURRENT_OFFSET_SCHEMA_VERSION = 1.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 OFFSET_COMMIT_VALUE_SCHEMA_V1 = new Schema(new Field("offset", INT64), - new Field("metadata", STRING, "Associated metadata.", ""), - new Field("commit_timestamp", INT64), - new Field("expire_timestamp", INT64)) - - private val VALUE_OFFSET_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") - private val VALUE_METADATA_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") - private val VALUE_TIMESTAMP_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") - - private val VALUE_OFFSET_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("offset") - private val VALUE_METADATA_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("metadata") - private val VALUE_COMMIT_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") - private val VALUE_EXPIRE_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") - - // map of versions to schemas - private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), - 1 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V1)) - - 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 - */ - private 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 - */ - private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { - // generate commit value with schema version 1 - val value = new Struct(CURRENT_SCHEMA.valueSchema) - value.set(VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) - value.set(VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) - value.set(VALUE_COMMIT_TIMESTAMP_FIELD_V1, offsetAndMetadata.commitTimestamp) - value.set(VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.expireTimestamp) - 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 - */ - private 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 - */ - private def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { - val structAndVersion = readMessageValueStruct(buffer) - - if (structAndVersion.value == null) { // tombstone - null - } else { - if (structAndVersion.version == 0) { - val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] - val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V0).asInstanceOf[String] - val timestamp = structAndVersion.value.get(VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] - - OffsetAndMetadata(offset, metadata, timestamp) - } else if (structAndVersion.version == 1) { - val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] - val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V1).asInstanceOf[String] - val commitTimestamp = structAndVersion.value.get(VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] - val expireTimestamp = structAndVersion.value.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] - - OffsetAndMetadata(offset, metadata, commitTimestamp, expireTimestamp) - } else { - throw new IllegalStateException("Unknown offset message version") - } - } - } - - private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { - if(buffer == null) { // tombstone - MessageValueStructAndVersion(null, -1) - } else { - val version = buffer.getShort() - val valueSchema = schemaFor(version).valueSchema - val value = valueSchema.read(buffer).asInstanceOf[Struct] - - MessageValueStructAndVersion(value, version) - } - } - - // 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.readMessageValueStruct(ByteBuffer.wrap(value)).value.toString - output.write(formattedKey.getBytes) - output.write("::".getBytes) - output.write(formattedValue.getBytes) - output.write("\n".getBytes) - } - } - -} - -case class MessageValueStructAndVersion(value: Struct, version: Short) - -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/package.html b/core/src/main/scala/kafka/server/package.html deleted file mode 100644 index 56ab9d41a8c9f..0000000000000 --- a/core/src/main/scala/kafka/server/package.html +++ /dev/null @@ -1,19 +0,0 @@ - -The kafka server. \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index db4295c49af73..9487c77d9fb79 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -193,7 +193,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { // get metadata for the topic var parts: Seq[PartitionInfo] = null while (parts == null) - parts = consumer0.partitionsFor(GroupCoordinator.OffsetsTopicName).asScala + parts = consumer0.partitionsFor(GroupCoordinator.GroupMetadataTopicName).asScala assertEquals(1, parts.size) assertNotNull(parts(0).leader()) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index bbc9a544fb3f7..b7ecc9d63232f 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -69,7 +69,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { } // create the consumer offset topic - TestUtils.createTopic(zkUtils, GroupCoordinator.OffsetsTopicName, + TestUtils.createTopic(zkUtils, GroupCoordinator.GroupMetadataTopicName, serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, servers, diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index 820a825288ab4..0df14c597ac55 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -85,12 +85,12 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging { // create the offset topic val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, "--replication-factor", "1", - "--topic", GroupCoordinator.OffsetsTopicName)) + "--topic", GroupCoordinator.GroupMetadataTopicName)) TopicCommand.createTopic(zkUtils, createOffsetTopicOpts) // try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't - val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", GroupCoordinator.OffsetsTopicName)) - val deleteOffsetTopicPath = getDeleteTopicPath(GroupCoordinator.OffsetsTopicName) + val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", GroupCoordinator.GroupMetadataTopicName)) + val deleteOffsetTopicPath = getDeleteTopicPath(GroupCoordinator.GroupMetadataTopicName) assertFalse("Delete path for topic shouldn't exist before deletion.", zkUtils.zkClient.exists(deleteOffsetTopicPath)) intercept[AdminOperationException] { TopicCommand.deleteTopic(zkUtils, deleteOffsetTopicOpts) diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 24fba45aa02b6..1e8d04eff1b1b 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -21,7 +21,6 @@ package kafka.consumer import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test -import kafka.server.OffsetManager import kafka.coordinator.GroupCoordinator @@ -38,8 +37,8 @@ class TopicFilterTest extends JUnitSuite { val topicFilter2 = new Whitelist(".+") assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true)) - assertFalse(topicFilter2.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter2.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter2.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter2.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = false)) val topicFilter3 = new Whitelist("white_listed-topic.+") assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true)) @@ -58,8 +57,8 @@ class TopicFilterTest extends JUnitSuite { assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) - assertFalse(topicFilter1.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter1.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = false)) } @Test From 17a83eb8529a57a97826473919764a02db1ca3b4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 15:10:50 -0700 Subject: [PATCH 02/23] add back package.html --- core/src/main/scala/kafka/server/package.html | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 core/src/main/scala/kafka/server/package.html diff --git a/core/src/main/scala/kafka/server/package.html b/core/src/main/scala/kafka/server/package.html new file mode 100644 index 0000000000000..56ab9d41a8c9f --- /dev/null +++ b/core/src/main/scala/kafka/server/package.html @@ -0,0 +1,19 @@ + +The kafka server. \ No newline at end of file From 1f2579d941fe25038738ec8e93f8b480ab7e7fe7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 16:28:32 -0700 Subject: [PATCH 03/23] persist synced assignment and client side error handling --- .../internals/AbstractCoordinator.java | 9 ++- .../internals/ConsumerCoordinator.java | 9 ++- .../apache/kafka/common/protocol/Errors.java | 4 +- .../common/requests/HeartbeatResponse.java | 1 + .../common/requests/JoinGroupResponse.java | 1 + .../common/requests/LeaveGroupResponse.java | 1 + .../common/requests/OffsetCommitResponse.java | 1 + .../common/requests/OffsetFetchResponse.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 2 +- .../kafka/common/OffsetMetadataAndError.scala | 2 +- .../coordinator/CoordinatorMetadata.scala | 81 ------------------- .../kafka/coordinator/GroupCoordinator.scala | 44 ++++++---- .../main/scala/kafka/server/KafkaConfig.scala | 19 ++--- .../coordinator/CoordinatorMetadataTest.scala | 71 ---------------- 14 files changed, 60 insertions(+), 187 deletions(-) delete mode 100644 core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala delete mode 100644 core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 4b2a824343485..de009dcbc8933 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -351,6 +351,10 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut } else { onJoinFollower().chain(future); } + } else if (errorCode == Errors.GROUP_LOAD_IN_PROGRESS.code()) { + log.debug("Attempt to join group {} rejected since coordinator is loading the group.", groupId); + // backoff and retry + future.raise(Errors.forCode(errorCode)); } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code()) { // reset the member id and retry immediately AbstractCoordinator.this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; @@ -362,7 +366,7 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut // re-discover the coordinator and retry with backoff coordinatorDead(); log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", - groupId); + groupId); future.raise(Errors.forCode(errorCode)); } else if (errorCode == Errors.INCONSISTENT_GROUP_PROTOCOL.code() || errorCode == Errors.INVALID_SESSION_TIMEOUT.code() @@ -538,6 +542,9 @@ public void handle(HeartbeatResponse heartbeatResponse, RequestFuture futu if (error == Errors.NONE.code()) { log.debug("Received successful heartbeat response."); future.complete(null); + } else if (error == Errors.GROUP_LOAD_IN_PROGRESS.code()) { + log.debug("Attempt to heart beat rejected since coordinator is loading the group, just treat it as successful."); + future.complete(null); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() || error == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 641939abf47d4..97d25c33a618d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -451,7 +451,10 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu // update the local cache only if the partition is still assigned subscriptions.committed(tp, offsetAndMetadata); } else { - if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() + if (errorCode == Errors.GROUP_LOAD_IN_PROGRESS.code()) { + // just retry + future.raise(Errors.GROUP_LOAD_IN_PROGRESS); + } else if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { coordinatorDead(); } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code() @@ -511,9 +514,9 @@ public void handle(OffsetFetchResponse response, RequestFuture groupConfig.groupMaxSessionTimeoutMs) { responseCallback(joinError(memberId, Errors.INVALID_SESSION_TIMEOUT.code)) @@ -126,12 +125,12 @@ class GroupCoordinator(val brokerId: Int, // only try to create the group if the group is not unknown AND // the member id is UNKNOWN, if member is specified but group does not // exist we should reject the request - var group = coordinatorMetadata.getGroup(groupId) + var group = groupManager.getGroup(groupId) if (group == null) { if (memberId != JoinGroupRequest.UNKNOWN_MEMBER_ID) { responseCallback(joinError(memberId, Errors.UNKNOWN_MEMBER_ID.code)) } else { - group = coordinatorMetadata.addGroup(groupId, protocolType) + group = groupManager.addGroup(groupId, protocolType) doJoinGroup(group, memberId, sessionTimeoutMs, protocolType, protocols, responseCallback) } } else { @@ -238,7 +237,7 @@ class GroupCoordinator(val brokerId: Int, } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Array.empty, Errors.NOT_COORDINATOR_FOR_GROUP.code) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) responseCallback(Array.empty, Errors.UNKNOWN_MEMBER_ID.code) else @@ -272,7 +271,9 @@ class GroupCoordinator(val brokerId: Int, // propagate the assignment to any awaiting members if (memberId == group.leaderId) { group.transitionTo(Stable) - propagateAssignment(group, groupAssignment) + + // persist the group metadata and upon finish propagate the assignment + groupManager.storeGroup(group, groupAssignment, propagateAssignment) } case Stable => @@ -290,8 +291,10 @@ class GroupCoordinator(val brokerId: Int, responseCallback(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code) } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code) + } else if (isCoordinatorLoadingInProgress(groupId)) { + responseCallback(Errors.GROUP_LOAD_IN_PROGRESS.code) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) { // if the group is marked as dead, it means some other thread has just removed the group // from the coordinator metadata; this is likely that the group has migrated to some other @@ -323,8 +326,10 @@ class GroupCoordinator(val brokerId: Int, responseCallback(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code) } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code) + } else if (isCoordinatorLoadingInProgress(groupId)) { + responseCallback(Errors.GROUP_LOAD_IN_PROGRESS.code) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) { // if the group is marked as dead, it means some other thread has just removed the group // from the coordinator metadata; this is likely that the group has migrated to some other @@ -360,8 +365,10 @@ class GroupCoordinator(val brokerId: Int, responseCallback(offsetMetadata.mapValues(_ => Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code)) } else if (!isCoordinatorForGroup(groupId)) { responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR_FOR_GROUP.code)) + } else if (isCoordinatorLoadingInProgress(groupId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.GROUP_LOAD_IN_PROGRESS.code)) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) { if (generationId < 0) // the group is not relying on Kafka for partition management, so allow the commit @@ -394,6 +401,8 @@ class GroupCoordinator(val brokerId: Int, partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.GroupCoordinatorNotAvailable)}.toMap } else if (!isCoordinatorForGroup(groupId)) { partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else if (isCoordinatorLoadingInProgress(groupId)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.GroupLoading)}.toMap } else { // return offsets blindly regardless the current group state since the group may be using // Kafka commit storage without automatic group management @@ -402,12 +411,10 @@ class GroupCoordinator(val brokerId: Int, } def handleGroupImmigration(offsetTopicPartitionId: Int) = { - // TODO we may need to add more logic in KAFKA-2017 groupManager.loadGroupsForPartition(offsetTopicPartitionId) } def handleGroupEmigration(offsetTopicPartitionId: Int) = { - // TODO we may need to add more logic in KAFKA-2017 groupManager.removeGroupsForPartition(offsetTopicPartitionId) } @@ -426,11 +433,12 @@ class GroupCoordinator(val brokerId: Int, } private def propagateAssignment(group: GroupMetadata, - assignment: Map[String, Array[Byte]]) { + assignment: Map[String, Array[Byte]], + errorCode: Short) { for (member <- group.allMembers) { member.assignment = assignment.getOrElse(member.memberId, Array.empty[Byte]) if (member.awaitingSyncCallback != null) { - member.awaitingSyncCallback(member.assignment, Errors.NONE.code) + member.awaitingSyncCallback(member.assignment, errorCode) member.awaitingSyncCallback = null } } @@ -540,7 +548,7 @@ class GroupCoordinator(val brokerId: Int, if (group.isEmpty) { group.transitionTo(Dead) info("Group %s generation %s is dead and removed".format(group.groupId, group.generationId)) - coordinatorMetadata.removeGroup(group.groupId) + groupManager.removeGroup(group.groupId) } } if (!group.is(Dead)) { @@ -592,7 +600,9 @@ class GroupCoordinator(val brokerId: Int, member.awaitingSyncCallback != null || member.latestHeartbeat + member.sessionTimeoutMs > heartbeatDeadline - private def isCoordinatorForGroup(groupId: String) = groupManager.leaderIsLocal(groupManager.partitionFor(groupId)) + private def isCoordinatorForGroup(groupId: String) = groupManager.partitionLeaderIsLocal(groupManager.partitionFor(groupId)) + + private def isCoordinatorLoadingInProgress(groupId: String) = groupManager.partitionLoadingInProgress(groupManager.partitionFor(groupId)) } object GroupCoordinator { diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index b749446bbe7f0..2973dab6b39ed 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -23,6 +23,7 @@ import java.util.Properties import kafka.api.ApiVersion import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig +import kafka.coordinator.OffsetConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs @@ -129,16 +130,16 @@ object Defaults { val ConsumerMaxSessionTimeoutMs = 30000 /** ********* Offset management configuration ***********/ - val OffsetMetadataMaxSize = OffsetManagerConfig.DefaultMaxMetadataSize - val OffsetsLoadBufferSize = OffsetManagerConfig.DefaultLoadBufferSize - val OffsetsTopicReplicationFactor = OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor - val OffsetsTopicPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions - val OffsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes - val OffsetsTopicCompressionCodec: Int = OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec.codec + val OffsetMetadataMaxSize = OffsetConfig.DefaultMaxMetadataSize + val OffsetsLoadBufferSize = OffsetConfig.DefaultLoadBufferSize + val OffsetsTopicReplicationFactor = OffsetConfig.DefaultOffsetsTopicReplicationFactor + val OffsetsTopicPartitions: Int = OffsetConfig.DefaultOffsetsTopicNumPartitions + val OffsetsTopicSegmentBytes: Int = OffsetConfig.DefaultOffsetsTopicSegmentBytes + val OffsetsTopicCompressionCodec: Int = OffsetConfig.DefaultOffsetsTopicCompressionCodec.codec val OffsetsRetentionMinutes: Int = 24 * 60 - val OffsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs - val OffsetCommitTimeoutMs = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs - val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks + val OffsetsRetentionCheckIntervalMs: Long = OffsetConfig.DefaultOffsetsRetentionCheckIntervalMs + val OffsetCommitTimeoutMs = OffsetConfig.DefaultOffsetCommitTimeoutMs + val OffsetCommitRequiredAcks = OffsetConfig.DefaultOffsetCommitRequiredAcks /** ********* Quota Configuration ***********/ val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala deleted file mode 100644 index 49a237bdbee74..0000000000000 --- a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala +++ /dev/null @@ -1,71 +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.coordinator - -import kafka.server.KafkaConfig -import kafka.utils.TestUtils - -import org.junit.Assert._ -import org.junit.{Before, Test} -import org.scalatest.junit.JUnitSuite - -/** - * Test coordinator group and topic metadata management - */ -class CoordinatorMetadataTest extends JUnitSuite { - val DefaultNumPartitions = 8 - val DefaultNumReplicas = 2 - var coordinatorMetadata: CoordinatorMetadata = null - - @Before - def setUp() { - val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") - coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId) - } - - @Test - def testGetNonexistentGroup() { - assertNull(coordinatorMetadata.getGroup("group")) - } - - @Test - def testGetGroup() { - val groupId = "group" - val protocolType = "consumer" - val expected = coordinatorMetadata.addGroup(groupId, protocolType) - val actual = coordinatorMetadata.getGroup(groupId) - assertEquals(expected, actual) - } - - @Test - def testAddGroupReturnsPreexistingGroupIfItAlreadyExists() { - val groupId = "group" - val protocolType = "consumer" - val group1 = coordinatorMetadata.addGroup(groupId, protocolType) - val group2 = coordinatorMetadata.addGroup(groupId, protocolType) - assertEquals(group1, group2) - } - - @Test(expected = classOf[IllegalArgumentException]) - def testRemoveNonexistentGroup() { - val groupId = "group" - val topics = Set("a") - coordinatorMetadata.removeGroup(groupId) - } - -} From 2dba3c1325892e1d4d5a81bf36b460376af73ffd Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 16:30:25 -0700 Subject: [PATCH 04/23] Add renamed files --- .../coordinator/GroupMetadataManager.scala | 842 ++++++++++++++++++ .../kafka/coordinator/OffsetConfig.scala | 61 ++ 2 files changed, 903 insertions(+) create mode 100644 core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala create mode 100644 core/src/main/scala/kafka/coordinator/OffsetConfig.scala diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala new file mode 100644 index 0000000000000..2b9564ee56c28 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -0,0 +1,842 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.coordinator + +import java.util.concurrent.locks.ReentrantReadWriteLock + +import kafka.utils.CoreUtils._ +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.types.{ArrayOf, 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 org.apache.kafka.common.protocol.types.Type.BYTES +import org.apache.kafka.common.utils.Utils + +import kafka.utils._ +import kafka.common._ +import kafka.message._ +import kafka.log.FileMessageSet +import kafka.metrics.KafkaMetricsGroup +import kafka.common.TopicAndPartition +import kafka.tools.MessageFormatter +import kafka.api.ProducerResponseStatus +import kafka.server.ReplicaManager + +import scala.collection._ +import java.io.PrintStream +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.TimeUnit + +import com.yammer.metrics.core.Gauge + +class GroupMetadataManager(val config: OffsetConfig, + replicaManager: ReplicaManager, + zkUtils: ZkUtils, + scheduler: Scheduler) extends Logging with KafkaMetricsGroup { + + /* offsets cache */ + private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata] + + /* group metadata cache */ + private val groupsCache = new Pool[String, GroupMetadata] + + /* partitions of consumer groups that are being loaded */ + private val loadingPartitions: mutable.Set[Int] = mutable.Set() + + /* lock for expiring stale offsets */ + private val offsetExpireLock = new ReentrantReadWriteLock() + + /* lock for removing offsets of a range partition */ + private val offsetRemoveLock = new ReentrantReadWriteLock() + + /* shutting down flag */ + private val shuttingDown = new AtomicBoolean(false) + + /* number of partitions for the consumer metadata topic */ + private val groupMetadataTopicPartitionCount = getOffsetsTopicPartitionCount + + this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " + + scheduler.schedule(name = "delete-expired-consumer-offsets", + fun = deleteExpiredOffsets, + 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 + } + ) + + def partitionFor(group: String): Int = Utils.abs(group.hashCode) % groupMetadataTopicPartitionCount + + def partitionLeaderIsLocal(partition: Int): Boolean = getHighWatermark(partition) != -1L + + def partitionLoadingInProgress(partition: Int): Boolean = loadingPartitions synchronized loadingPartitions.contains(partition) + + /** + * Get the group associated with the given groupId, or null if not found + */ + def getGroup(groupId: String) = { + groupsCache.get(groupId) + } + + /** + * Add a group or get the group associated with the given groupId if it already exists + */ + def addGroup(groupId: String, protocolType: String): GroupMetadata = { + addGroup(groupId, new GroupMetadata(groupId, protocolType)) + } + + private def addGroup(groupId: String, group: GroupMetadata): GroupMetadata = { + groupsCache.putIfNotExists(groupId, group) + } + + /** + * Remove all metadata associated with the group, including its topics + * @param groupId the groupId of the group we are removing + */ + def removeGroup(groupId: String) { + if (!groupsCache.contains(groupId)) + throw new IllegalArgumentException("Cannot remove non-existing group") + groupsCache.remove(groupId) + } + + def storeGroup(group: GroupMetadata, + groupAssignment: Map[String, Array[Byte]], + responseCallback: (GroupMetadata, Map[String, Array[Byte]], Short) => Unit) { + // construct the message to append + val message = new Message( + key = GroupMetadataManager.groupMetadataKey(group.groupId), + bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment) + ) + + val groupMetadataPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(group.groupId)) + + val groupMetadataMessageSet = Map(groupMetadataPartition -> + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message)) + + // set the callback function to insert offsets into cache after log append completed + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + // the append response should only contain the topics partition + if (responseStatus.size != 1 || ! responseStatus.contains(groupMetadataPartition)) + throw new IllegalStateException("Append status %s should only have one partition %s" + .format(responseStatus, groupMetadataPartition)) + + // construct the commit response status and insert + // the offset and metadata to cache if the append status has no error + val status = responseStatus(groupMetadataPartition) + + val responseCode = + if (status.error != ErrorMapping.NoError) { + debug("Metadata from group %s with generation %d failed when appending to log due to %s" + .format(group.groupId, group.generationId, ErrorMapping.exceptionNameFor(status.error))) + + // transform the log append error code to the corresponding the commit status error code + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code + else + status.error + } else { + Errors.NONE.code + } + + // finally trigger the callback logic passed from the API layer + responseCallback(group, groupAssignment, responseCode) + } + + // call replica manager to append the offset messages + replicaManager.appendMessages( + config.offsetCommitTimeoutMs.toLong, + config.offsetCommitRequiredAcks, + true, // allow appending to internal offset topic + groupMetadataMessageSet, + putCacheCallback) + } + + /** + * Store offsets by appending it to the replicated log and then inserting to cache + */ + def storeOffsets(groupId: String, + consumerId: String, + generationId: Int, + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + // first filter out partitions with offset metadata size exceeding limit + val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => + validateOffsetMetadataLength(offsetAndMetadata.metadata) + } + + // construct the message set to append + val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + new Message( + key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), + bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata) + ) + }.toSeq + + val offsetTopicPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(groupId)) + + val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) + + // set the callback function to insert offsets into cache after log append completed + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + // the append response should only contain the topics partition + if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) + throw new IllegalStateException("Append status %s should only have one partition %s" + .format(responseStatus, offsetTopicPartition)) + + // construct the commit response status and insert + // the offset and metadata to cache if the append status has no error + val status = responseStatus(offsetTopicPartition) + + val responseCode = + if (status.error == ErrorMapping.NoError) { + filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => + putOffset(GroupTopicPartition(groupId, topicAndPartition), offsetAndMetadata) + } + ErrorMapping.NoError + } else { + debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" + .format(filteredOffsetMetadata, groupId, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) + + // transform the log append error code to the corresponding the commit status error code + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code + else + status.error + } + + + // compute the final error codes for the commit response + val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) + (topicAndPartition, responseCode) + else + (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) + } + + // finally trigger the callback logic passed from the API layer + responseCallback(commitStatus) + } + + // call replica manager to append the offset messages + replicaManager.appendMessages( + config.offsetCommitTimeoutMs.toLong, + config.offsetCommitRequiredAcks, + true, // allow appending to internal offset topic + offsetsAndMetadataMessageSet, + putCacheCallback) + } + + /** + * 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) + + /** + * we need to put the leader-is-local check inside the offsetLock to protects against fetching from an empty/cleared + * offset cache (i.e., cleared due to a leader->follower transition right after the check and clear the cache). + */ + inReadLock(offsetRemoveLock) { + if (partitionLeaderIsLocal(offsetsPartition)) { + if (topicPartitions.isEmpty) { + // 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.NotCoordinatorForGroup) + }.toMap + } + } + } + + /** + * Asynchronously read the partition from the offsets topic and populate the cache + */ + def loadGroupsForPartition(offsetsPartition: Int) { + + val topicPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, 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, loadGroupsAndOffsets) + } + } + + def loadGroupsAndOffsets() { + 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 + inWriteLock(offsetExpireLock) { + while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { + buffer.clear() + val messages = log.read(currOffset, config.loadBufferSize).messageSet.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 baseKey = GroupMetadataManager.readMessageKey(msgAndOffset.message.key) + + if (baseKey.isInstanceOf[OffsetKey]) { + // load offset + val key = baseKey.key.asInstanceOf[GroupTopicPartition] + 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 { + // special handling for version 0: + // set the expiration time stamp as commit time stamp + server default retention time + val value = GroupMetadataManager.readOffsetMessageValue(msgAndOffset.message.payload) + putOffset(key, value.copy ( + expireTimestamp = { + if (value.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + value.commitTimestamp + config.offsetsRetentionMs + else + value.expireTimestamp + } + )) + trace("Loaded offset %s for %s.".format(value, key)) + } + } else { + // load group metadata + val groupId = baseKey.key.asInstanceOf[String] + val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, msgAndOffset.message.payload) + + addGroup(groupId, groupMetadata) + } + + 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) + } + } + } + + /** + * 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 removeGroupsForPartition(offsetsPartition: Int) { + var numRemoved = 0 + inWriteLock(offsetRemoveLock) { + offsetsCache.keys.foreach { key => + if (partitionFor(key.group) == offsetsPartition) { + offsetsCache.remove(key) + numRemoved += 1 + } + } + + // TODO: we may also remove the group metadata cache here + } + + if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." + .format(numRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) + } + + /** + * 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) + } + + private def deleteExpiredOffsets() { + debug("Collecting expired offsets.") + val startMs = SystemTime.milliseconds + + val numExpiredOffsetsRemoved = inWriteLock(offsetExpireLock) { + val expiredOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => + offsetAndMetadata.expireTimestamp < startMs + } + + debug("Found %d expired offsets.".format(expiredOffsets.size)) + + // delete the expired offsets from the table and generate tombstone messages to remove them from the log + val tombstonesForPartition = expiredOffsets.map { case (groupTopicAndPartition, offsetAndMetadata) => + val offsetsPartition = partitionFor(groupTopicAndPartition.group) + trace("Removing expired offset and metadata for %s: %s".format(groupTopicAndPartition, offsetAndMetadata)) + + offsetsCache.remove(groupTopicAndPartition) + + val commitKey = GroupMetadataManager.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 expired offsets during their own purge cycles. + tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) => + val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition) + partitionOpt.map { partition => + val appendPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition) + val messages = tombstones.map(_._2).toSeq + + trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) + + try { + // do not need to require acks since even if the tombsone is lost, + // it will be appended again in the next purge cycle + partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages: _*)) + tombstones.size + } + catch { + case t: Throwable => + error("Failed to mark %d expired offsets for deletion in %s.".format(messages.size, appendPartition), t) + // ignore and continue + 0 + } + } + }.sum + } + + info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, SystemTime.milliseconds - startMs)) + } + + private def getHighWatermark(partitionId: Int): Long = { + val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, partitionId) + + val hw = partitionOpt.map { partition => + partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) + }.getOrElse(-1L) + + hw + } + + /* + * Check if the offset metadata length is valid + */ + private def validateOffsetMetadataLength(metadata: String) : Boolean = { + metadata == null || metadata.length() <= config.maxMetadataSize + } + + def shutdown() { + shuttingDown.set(true) + + // TODO: clear the caches + } + + /** + * Gets the partition count of the offsets topic from ZooKeeper. + * If the topic does not exist, the configured partition count is returned. + */ + private def getOffsetsTopicPartitionCount = { + val topic = GroupCoordinator.GroupMetadataTopicName + val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic)) + if (topicData(topic).nonEmpty) + topicData(topic).size + else + config.offsetsTopicNumPartitions + } +} + + +object GroupMetadataManager { + + private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) + + private val CURRENT_OFFSET_SCHEMA_VERSION = 1.toShort + + private val CURRENT_GROUP_SCHEMA_VERSION = 2.toShort + + private val OFFSET_COMMIT_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING), + new Field("topic", STRING), + new Field("partition", INT32)) + private val OFFSET_KEY_GROUP_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("group") + private val OFFSET_KEY_TOPIC_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("topic") + private val OFFSET_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 OFFSET_VALUE_OFFSET_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") + private val OFFSET_VALUE_METADATA_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") + private val OFFSET_VALUE_TIMESTAMP_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") + + + private val OFFSET_COMMIT_VALUE_SCHEMA_V1 = new Schema(new Field("offset", INT64), + new Field("metadata", STRING, "Associated metadata.", ""), + new Field("commit_timestamp", INT64), + new Field("expire_timestamp", INT64)) + private val OFFSET_VALUE_OFFSET_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("offset") + private val OFFSET_VALUE_METADATA_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("metadata") + private val OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") + private val OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") + + private val GROUP_METADATA_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING)) + private val GROUP_KEY_GROUP_FIELD = GROUP_METADATA_KEY_SCHEMA_V0.get("group") + + private val MEMBER_METADATA_V0 = new Schema(new Field("member_id", STRING), + new Field("session_timeout", INT32), + new Field("subscription", BYTES), + new Field("assignment", BYTES)) + private val MEMBER_METADATA_MEMBER_ID_V0 = MEMBER_METADATA_V0.get("member_id") + private val MEMBER_METADATA_SESSION_TIMEOUT_V0 = MEMBER_METADATA_V0.get("session_timeout") + private val MEMBER_METADATA_SUBSCRIPTION_V0 = MEMBER_METADATA_V0.get("subscription") + private val MEMBER_METADATA_ASSIGNMENT_V0 = MEMBER_METADATA_V0.get("assignment") + + + private val GROUP_METADATA_VALUE_SCHEMA_V0 = new Schema(new Field("protocol_type", STRING), + new Field("generation", INT32), + new Field("protocol", STRING), + new Field("leader", STRING), + new Field("members", new ArrayOf(MEMBER_METADATA_V0))) + private val GROUP_METADATA_PROTOCOL_TYPE_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("protocol_type") + private val GROUP_METADATA_GENERATION_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("generation") + private val GROUP_METADATA_PROTOCOL_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("protocol") + private val GROUP_METADATA_LEADER_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("leader") + private val GROUP_METADATA_MEMBERS_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("members") + + // map of versions to schemas + private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), + 1 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V1), + 2 -> KeyAndValueSchemas(GROUP_METADATA_KEY_SCHEMA_V0, GROUP_METADATA_VALUE_SCHEMA_V0)) + + private val CURRENT_OFFSET_SCHEMA = schemaFor(CURRENT_OFFSET_SCHEMA_VERSION) + private val CURRENT_GROUP_SCHEMA = schemaFor(CURRENT_GROUP_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 + */ + private def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { + val key = new Struct(CURRENT_OFFSET_SCHEMA.keySchema) + key.set(OFFSET_KEY_GROUP_FIELD, group) + key.set(OFFSET_KEY_TOPIC_FIELD, topic) + key.set(OFFSET_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 key for group metadata message for given group + * + * @return key bytes for group metadata message + */ + private def groupMetadataKey(group: String): Array[Byte] = { + val key = new Struct(CURRENT_GROUP_SCHEMA.keySchema) + key.set(GROUP_KEY_GROUP_FIELD, group) + + val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf) + byteBuffer.putShort(CURRENT_GROUP_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 + */ + private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { + // generate commit value with schema version 1 + val value = new Struct(CURRENT_OFFSET_SCHEMA.valueSchema) + value.set(OFFSET_VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) + value.set(OFFSET_VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) + value.set(OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1, offsetAndMetadata.commitTimestamp) + value.set(OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.expireTimestamp) + val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) + byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + value.writeTo(byteBuffer) + byteBuffer.array() + } + + /** + * Generates the payload for group metadata message from given offset and metadata + * assuming the generation id, selected protocol, leader and member assignment are all available + * + * @param groupMetadata + * @return payload for offset commit message + */ + private def groupMetadataValue(groupMetadata: GroupMetadata, assignment: Map[String, Array[Byte]]): Array[Byte] = { + // generate commit value with schema version 1 + val value = new Struct(CURRENT_GROUP_SCHEMA.valueSchema) + value.set(GROUP_METADATA_PROTOCOL_TYPE_V0, groupMetadata.protocolType) + value.set(GROUP_METADATA_GENERATION_V0, groupMetadata.generationId) + value.set(GROUP_METADATA_PROTOCOL_V0, groupMetadata.protocol) + value.set(GROUP_METADATA_LEADER_V0, groupMetadata.leaderId) + + val memberArray = groupMetadata.allMembers.map { + case memberMetadata => + val memberStruct = value.instance(GROUP_METADATA_MEMBERS_V0) + memberStruct.set(MEMBER_METADATA_MEMBER_ID_V0, memberMetadata.memberId) + memberStruct.set(MEMBER_METADATA_SESSION_TIMEOUT_V0, memberMetadata.sessionTimeoutMs) + + val selectedProtocol = memberMetadata.supportedProtocols.filter(_._1.equals(groupMetadata.protocol)) + assert(selectedProtocol.length == 1) + + memberStruct.set(MEMBER_METADATA_SUBSCRIPTION_V0, selectedProtocol.head._2) + + val memberAssignment = assignment(memberMetadata.memberId) + assert(memberAssignment != null) + + memberStruct.set(MEMBER_METADATA_ASSIGNMENT_V0, memberAssignment) + + memberStruct + } + + value.set(GROUP_METADATA_MEMBERS_V0, memberArray.toArray) + + 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 + */ + private def readMessageKey(buffer: ByteBuffer): BaseKey = { + val version = buffer.getShort + val keySchema = schemaFor(version).keySchema + val key = keySchema.read(buffer).asInstanceOf[Struct] + + if (version <= CURRENT_OFFSET_SCHEMA_VERSION) { + // version 0 and 1 refer to offset + val group = key.get(OFFSET_KEY_GROUP_FIELD).asInstanceOf[String] + val topic = key.get(OFFSET_KEY_TOPIC_FIELD).asInstanceOf[String] + val partition = key.get(OFFSET_KEY_PARTITION_FIELD).asInstanceOf[Int] + + OffsetKey(version, GroupTopicPartition(group, TopicAndPartition(topic, partition))) + + } else if (version == CURRENT_GROUP_SCHEMA_VERSION) { + // version 2 refers to offset + val group = key.get(GROUP_KEY_GROUP_FIELD).asInstanceOf[String] + + GroupKey(version, group) + } else { + throw new IllegalStateException("Unknown version " + version + " for group metadata message") + } + } + + /** + * 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 + */ + private def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { + val structAndVersion = readMessageValueStruct(buffer) + + if (structAndVersion.value == null) { // tombstone + null + } else { + if (structAndVersion.version == 0) { + val offset = structAndVersion.value.get(OFFSET_VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] + val metadata = structAndVersion.value.get(OFFSET_VALUE_METADATA_FIELD_V0).asInstanceOf[String] + val timestamp = structAndVersion.value.get(OFFSET_VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, timestamp) + } else if (structAndVersion.version == 1) { + val offset = structAndVersion.value.get(OFFSET_VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] + val metadata = structAndVersion.value.get(OFFSET_VALUE_METADATA_FIELD_V1).asInstanceOf[String] + val commitTimestamp = structAndVersion.value.get(OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + val expireTimestamp = structAndVersion.value.get(OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, commitTimestamp, expireTimestamp) + } else { + throw new IllegalStateException("Unknown offset message version") + } + } + } + + /** + * Decodes the group metadata messages' payload and retrieves its member metadatafrom it + * + * @param buffer input byte-buffer + * @return a group metadata object from the message + */ + private def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = { + val structAndVersion = readMessageValueStruct(buffer) + + if (structAndVersion.value == null) { // tombstone + null + } else { + if (structAndVersion.version == CURRENT_GROUP_SCHEMA_VERSION) { + val protocolType = structAndVersion.value.get(GROUP_METADATA_PROTOCOL_TYPE_V0).asInstanceOf[String] + + val group = new GroupMetadata(groupId, protocolType) + + group.generationId = structAndVersion.value.get(GROUP_METADATA_GENERATION_V0).asInstanceOf[Int] + group.leaderId = structAndVersion.value.get(GROUP_METADATA_LEADER_V0).asInstanceOf[String] + group.protocol = structAndVersion.value.get(GROUP_METADATA_PROTOCOL_V0).asInstanceOf[String] + + structAndVersion.value.getArray(GROUP_METADATA_MEMBERS_V0).foreach { + case memberMetadataObj => + val memberMetadata = memberMetadataObj.asInstanceOf[Struct] + val memberId = memberMetadata.get(MEMBER_METADATA_MEMBER_ID_V0).asInstanceOf[String] + val sessionTimeout = memberMetadata.get(MEMBER_METADATA_SESSION_TIMEOUT_V0).asInstanceOf[Int] + val subscription = memberMetadata.get(MEMBER_METADATA_SUBSCRIPTION_V0).asInstanceOf[Array[Byte]] + + val member = new MemberMetadata(memberId, groupId, sessionTimeout, List((group.protocol, subscription))) + + member.assignment = memberMetadata.get(MEMBER_METADATA_ASSIGNMENT_V0).asInstanceOf[Array[Byte]] + + group.add(memberId, member) + } + + group + } else { + throw new IllegalStateException("Unknown group metadata message version") + } + } + } + + private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { + if(buffer == null) { // tombstone + MessageValueStructAndVersion(null, -1) + } else { + val version = buffer.getShort + val valueSchema = schemaFor(version).valueSchema + val value = valueSchema.read(buffer).asInstanceOf[Struct] + + MessageValueStructAndVersion(value, version) + } + } + + // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. + // (specify --formatter "kafka.coordinator.GroupMetadataManager\$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 GroupMetadataManager.readMessageKey(ByteBuffer.wrap(key)).toString + val formattedValue = if (value == null) "NULL" else GroupMetadataManager.readMessageValueStruct(ByteBuffer.wrap(value)).value.toString + output.write(formattedKey.getBytes) + output.write("::".getBytes) + output.write(formattedValue.getBytes) + output.write("\n".getBytes) + } + } + +} + +case class MessageValueStructAndVersion(value: Struct, version: Short) + +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) +} + +trait BaseKey{ + def version: Short + def key: Object +} + +case class OffsetKey(version: Short, key: GroupTopicPartition) extends BaseKey + +case class GroupKey(version: Short, key: String) extends BaseKey + diff --git a/core/src/main/scala/kafka/coordinator/OffsetConfig.scala b/core/src/main/scala/kafka/coordinator/OffsetConfig.scala new file mode 100644 index 0000000000000..92f56b2076883 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/OffsetConfig.scala @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.coordinator + +import kafka.message.{NoCompressionCodec, CompressionCodec} + +/** + * 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 expired 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 OffsetConfig(maxMetadataSize: Int = OffsetConfig.DefaultMaxMetadataSize, + loadBufferSize: Int = OffsetConfig.DefaultLoadBufferSize, + offsetsRetentionMs: Long = OffsetConfig.DefaultOffsetRetentionMs, + offsetsRetentionCheckIntervalMs: Long = OffsetConfig.DefaultOffsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions: Int = OffsetConfig.DefaultOffsetsTopicNumPartitions, + offsetsTopicSegmentBytes: Int = OffsetConfig.DefaultOffsetsTopicSegmentBytes, + offsetsTopicReplicationFactor: Short = OffsetConfig.DefaultOffsetsTopicReplicationFactor, + offsetsTopicCompressionCodec: CompressionCodec = OffsetConfig.DefaultOffsetsTopicCompressionCodec, + offsetCommitTimeoutMs: Int = OffsetConfig.DefaultOffsetCommitTimeoutMs, + offsetCommitRequiredAcks: Short = OffsetConfig.DefaultOffsetCommitRequiredAcks) + +object OffsetConfig { + val DefaultMaxMetadataSize = 4096 + val DefaultLoadBufferSize = 5*1024*1024 + val DefaultOffsetRetentionMs = 24*60*60*1000L + val DefaultOffsetsRetentionCheckIntervalMs = 600000L + val DefaultOffsetsTopicNumPartitions = 50 + val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 + val DefaultOffsetsTopicReplicationFactor = 3.toShort + val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec + val DefaultOffsetCommitTimeoutMs = 5000 + val DefaultOffsetCommitRequiredAcks = (-1).toShort +} \ No newline at end of file From fb6f49b494200cb974001953431a73f354dd513d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 16:42:30 -0700 Subject: [PATCH 05/23] bindly accepts HB while loading in progress --- .../kafka/clients/consumer/internals/AbstractCoordinator.java | 3 --- .../org/apache/kafka/common/requests/HeartbeatResponse.java | 1 - core/src/main/scala/kafka/coordinator/GroupCoordinator.scala | 3 ++- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index de009dcbc8933..d8f3c25ec907b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -542,9 +542,6 @@ public void handle(HeartbeatResponse heartbeatResponse, RequestFuture futu if (error == Errors.NONE.code()) { log.debug("Received successful heartbeat response."); future.complete(null); - } else if (error == Errors.GROUP_LOAD_IN_PROGRESS.code()) { - log.debug("Attempt to heart beat rejected since coordinator is loading the group, just treat it as successful."); - future.complete(null); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() || error == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index 1345e35425865..48cb4c0f016d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -27,7 +27,6 @@ public class HeartbeatResponse extends AbstractRequestResponse { /** * Possible error code: * - * GROUP_LOAD_IN_PROGRESS (14) * GROUP_COORDINATOR_NOT_AVAILABLE (15) * NOT_COORDINATOR_FOR_GROUP (16) * ILLEGAL_GENERATION (22) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 8c393de9e048c..efde1b709e254 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -327,7 +327,8 @@ class GroupCoordinator(val brokerId: Int, } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code) } else if (isCoordinatorLoadingInProgress(groupId)) { - responseCallback(Errors.GROUP_LOAD_IN_PROGRESS.code) + // the group is still loading, so respond just blindly + responseCallback(Errors.NONE.code) } else { val group = groupManager.getGroup(groupId) if (group == null) { From 0edb0ec9fbf1f51a7e2123c604ca605b732a3c53 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 15:07:17 -0700 Subject: [PATCH 06/23] move group metadata into offset manager --- .../main/scala/kafka/admin/TopicCommand.scala | 2 +- core/src/main/scala/kafka/common/Topic.scala | 2 +- .../kafka/coordinator/GroupCoordinator.scala | 36 +- .../main/scala/kafka/server/KafkaApis.scala | 10 +- .../scala/kafka/server/OffsetManager.scala | 627 ------------------ core/src/main/scala/kafka/server/package.html | 19 - .../kafka/api/BaseConsumerTest.scala | 2 +- .../kafka/api/IntegrationTestHarness.scala | 2 +- .../unit/kafka/admin/TopicCommandTest.scala | 6 +- .../unit/kafka/consumer/TopicFilterTest.scala | 9 +- 10 files changed, 35 insertions(+), 680 deletions(-) delete mode 100755 core/src/main/scala/kafka/server/OffsetManager.scala delete mode 100644 core/src/main/scala/kafka/server/package.html diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index ed54aee39c66b..8686768978f56 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -129,7 +129,7 @@ object TopicCommand extends Logging { } if(opts.options.has(opts.partitionsOpt)) { - if (topic == GroupCoordinator.OffsetsTopicName) { + if (topic == GroupCoordinator.GroupMetadataTopicName) { 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 " + diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index ca41eba77a4bd..982955e9ae977 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -26,7 +26,7 @@ object Topic { private val maxNameLength = 255 private val rgx = new Regex(legalChars + "+") - val InternalTopics = Set(GroupCoordinator.OffsetsTopicName) + val InternalTopics = Set(GroupCoordinator.GroupMetadataTopicName) def validate(topic: String) { if (topic.length <= 0) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 0ef542d485d8a..8fb69efb2f992 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -47,8 +47,8 @@ case class JoinGroupResult(members: Map[String, Array[Byte]], */ class GroupCoordinator(val brokerId: Int, val groupConfig: GroupManagerConfig, - val offsetConfig: OffsetManagerConfig, - private val offsetManager: OffsetManager) extends Logging { + val offsetConfig: OffsetConfig, + private val groupManager: GroupMetadataManager) extends Logging { type JoinCallback = JoinGroupResult => Unit type SyncCallback = (Array[Byte], Short) => Unit @@ -62,11 +62,11 @@ class GroupCoordinator(val brokerId: Int, def this(brokerId: Int, groupConfig: GroupManagerConfig, - offsetConfig: OffsetManagerConfig, + offsetConfig: OffsetConfig, replicaManager: ReplicaManager, zkUtils: ZkUtils, scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig, - new OffsetManager(offsetConfig, replicaManager, zkUtils, scheduler)) + new GroupMetadataManager(offsetConfig, replicaManager, zkUtils, scheduler)) def offsetsTopicConfigs: Properties = { val props = new Properties @@ -100,7 +100,7 @@ class GroupCoordinator(val brokerId: Int, def shutdown() { info("Shutting down.") isActive.set(false) - offsetManager.shutdown() + groupManager.shutdown() coordinatorMetadata.shutdown() heartbeatPurgatory.shutdown() joinPurgatory.shutdown() @@ -365,7 +365,7 @@ class GroupCoordinator(val brokerId: Int, if (group == null) { if (generationId < 0) // the group is not relying on Kafka for partition management, so allow the commit - offsetManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) + groupManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) else // the group has failed over to this coordinator (which will be handled in KAFKA-2017), // or this is a request coming from an older generation. either way, reject the commit @@ -381,7 +381,7 @@ class GroupCoordinator(val brokerId: Int, } else if (generationId != group.generationId) { responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code)) } else { - offsetManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) + groupManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback) } } } @@ -397,18 +397,18 @@ class GroupCoordinator(val brokerId: Int, } else { // return offsets blindly regardless the current group state since the group may be using // Kafka commit storage without automatic group management - offsetManager.getOffsets(groupId, partitions) + groupManager.getOffsets(groupId, partitions) } } def handleGroupImmigration(offsetTopicPartitionId: Int) = { // TODO we may need to add more logic in KAFKA-2017 - offsetManager.loadOffsetsFromLog(offsetTopicPartitionId) + groupManager.loadGroupsForPartition(offsetTopicPartitionId) } def handleGroupEmigration(offsetTopicPartitionId: Int) = { // TODO we may need to add more logic in KAFKA-2017 - offsetManager.removeOffsetsFromCacheForPartition(offsetTopicPartitionId) + groupManager.removeGroupsForPartition(offsetTopicPartitionId) } private def validGroupId(groupId: String): Boolean = { @@ -585,27 +585,29 @@ class GroupCoordinator(val brokerId: Int, // TODO: add metrics for complete heartbeats } - def partitionFor(group: String): Int = offsetManager.partitionFor(group) + def partitionFor(group: String): Int = groupManager.partitionFor(group) private def shouldKeepMemberAlive(member: MemberMetadata, heartbeatDeadline: Long) = member.awaitingJoinCallback != null || member.awaitingSyncCallback != null || member.latestHeartbeat + member.sessionTimeoutMs > heartbeatDeadline - private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId)) + private def isCoordinatorForGroup(groupId: String) = groupManager.leaderIsLocal(groupManager.partitionFor(groupId)) } object GroupCoordinator { val NoProtocol = "" val NoLeader = "" - val OffsetsTopicName = "__consumer_offsets" + + // TODO: we store both group metadata and offset data here despite the topic name being offsets only + val GroupMetadataTopicName = "__consumer_offsets" def create(config: KafkaConfig, zkUtils: ZkUtils, replicaManager: ReplicaManager, kafkaScheduler: KafkaScheduler): GroupCoordinator = { - val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize, loadBufferSize = config.offsetsLoadBufferSize, offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, @@ -621,8 +623,8 @@ object GroupCoordinator { def create(config: KafkaConfig, zkUtils: ZkUtils, - offsetManager: OffsetManager): GroupCoordinator = { - val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize, + groupManager: GroupMetadataManager): GroupCoordinator = { + val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize, loadBufferSize = config.offsetsLoadBufferSize, offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, @@ -633,6 +635,6 @@ object GroupCoordinator { val groupConfig = GroupManagerConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs) - new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager) + new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index dfcfa25e09152..c234558060ca6 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -116,12 +116,12 @@ class KafkaApis(val requestChannel: RequestChannel, // for each new leader or follower, call coordinator to handle // consumer group migration result.updatedLeaders.foreach { case partition => - if (partition.topic == GroupCoordinator.OffsetsTopicName) + if (partition.topic == GroupCoordinator.GroupMetadataTopicName) coordinator.handleGroupImmigration(partition.partitionId) } result.updatedFollowers.foreach { case partition => partition.leaderReplicaIdOpt.foreach { leaderReplica => - if (partition.topic == GroupCoordinator.OffsetsTopicName && + if (partition.topic == GroupCoordinator.GroupMetadataTopicName && leaderReplica == brokerId) coordinator.handleGroupEmigration(partition.partitionId) } @@ -546,9 +546,9 @@ class KafkaApis(val requestChannel: RequestChannel, if (topics.size > 0 && topicResponses.size != topics.size) { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => - if (topic == GroupCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) { + if (topic == GroupCoordinator.GroupMetadataTopicName || config.autoCreateTopicsEnable) { try { - if (topic == GroupCoordinator.OffsetsTopicName) { + if (topic == GroupCoordinator.GroupMetadataTopicName) { val aliveBrokers = metadataCache.getAliveBrokers val offsetsTopicReplicationFactor = if (aliveBrokers.length > 0) @@ -689,7 +689,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = coordinator.partitionFor(groupMetadataRequest.group) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(GroupCoordinator.OffsetsTopicName), request.securityProtocol).head + val offsetsTopicMetadata = getTopicMetadata(Set(GroupCoordinator.GroupMetadataTopicName), request.securityProtocol).head val errorResponse = GroupMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, groupMetadataRequest.correlationId) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala deleted file mode 100755 index 967dc6fa955cd..0000000000000 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ /dev/null @@ -1,627 +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.server - -import org.apache.kafka.common.protocol.Errors -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 org.apache.kafka.common.utils.Utils - -import kafka.utils._ -import kafka.common._ -import kafka.log.FileMessageSet -import kafka.message._ -import kafka.metrics.KafkaMetricsGroup -import kafka.common.TopicAndPartition -import kafka.tools.MessageFormatter -import kafka.api.ProducerResponseStatus -import kafka.coordinator.GroupCoordinator - -import scala.Some -import scala.collection._ -import java.io.PrintStream -import java.util.concurrent.atomic.AtomicBoolean -import java.nio.ByteBuffer -import java.util.concurrent.TimeUnit - -import com.yammer.metrics.core.Gauge -import org.I0Itec.zkclient.ZkClient - -/** - * 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 expired 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 = OffsetManagerConfig.DefaultOffsetRetentionMs, - 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 DefaultOffsetRetentionMs = 24*60*60*1000L - val DefaultOffsetsRetentionCheckIntervalMs = 600000L - val DefaultOffsetsTopicNumPartitions = 50 - val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 - val DefaultOffsetsTopicReplicationFactor = 3.toShort - val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec - val DefaultOffsetCommitTimeoutMs = 5000 - val DefaultOffsetCommitRequiredAcks = (-1).toShort -} - -class OffsetManager(val config: OffsetManagerConfig, - replicaManager: ReplicaManager, - zkUtils: ZkUtils, - 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 cleanupOrLoadMutex = new Object - private val shuttingDown = new AtomicBoolean(false) - private val offsetsTopicPartitionCount = getOffsetsTopicPartitionCount - - this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " - - scheduler.schedule(name = "delete-expired-consumer-offsets", - fun = deleteExpiredOffsets, - 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 deleteExpiredOffsets() { - debug("Collecting expired offsets.") - val startMs = SystemTime.milliseconds - - val numExpiredOffsetsRemoved = cleanupOrLoadMutex synchronized { - val expiredOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => - offsetAndMetadata.expireTimestamp < startMs - } - - debug("Found %d expired offsets.".format(expiredOffsets.size)) - - // delete the expired offsets from the table and generate tombstone messages to remove them from the log - val tombstonesForPartition = expiredOffsets.map { case (groupTopicAndPartition, offsetAndMetadata) => - val offsetsPartition = partitionFor(groupTopicAndPartition.group) - trace("Removing expired 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 expired offsets during their own purge cycles. - tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) => - val partitionOpt = replicaManager.getPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition) - partitionOpt.map { partition => - val appendPartition = TopicAndPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition) - val messages = tombstones.map(_._2).toSeq - - trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) - - try { - // do not need to require acks since even if the tombsone is lost, - // it will be appended again in the next purge cycle - partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages: _*)) - tombstones.size - } - catch { - case t: Throwable => - error("Failed to mark %d expired offsets for deletion in %s.".format(messages.size, appendPartition), t) - // ignore and continue - 0 - } - } - }.sum - } - - info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, SystemTime.milliseconds - startMs)) - } - - - def partitionFor(group: String): Int = Utils.abs(group.hashCode) % offsetsTopicPartitionCount - - /** - * 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) - } - - /* - * Check if the offset metadata length is valid - */ - def validateOffsetMetadataLength(metadata: String) : Boolean = { - metadata == null || metadata.length() <= config.maxMetadataSize - } - - /** - * Store offsets by appending it to the replicated log and then inserting to cache - */ - def storeOffsets(groupId: String, - consumerId: String, - generationId: Int, - offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], - responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { - // first filter out partitions with offset metadata size exceeding limit - val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - validateOffsetMetadataLength(offsetAndMetadata.metadata) - } - - // construct the message set to append - val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - new Message( - key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) - ) - }.toSeq - - val offsetTopicPartition = TopicAndPartition(GroupCoordinator.OffsetsTopicName, partitionFor(groupId)) - - val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> - new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) - - // set the callback function to insert offsets into cache after log append completed - def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { - // the append response should only contain the topics partition - if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) - throw new IllegalStateException("Append status %s should only have one partition %s" - .format(responseStatus, offsetTopicPartition)) - - // construct the commit response status and insert - // the offset and metadata to cache if the append status has no error - val status = responseStatus(offsetTopicPartition) - - val responseCode = - if (status.error == ErrorMapping.NoError) { - filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(groupId, topicAndPartition), offsetAndMetadata) - } - ErrorMapping.NoError - } else { - debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" - .format(filteredOffsetMetadata, groupId, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) - - // transform the log append error code to the corresponding the commit status error code - if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) - ErrorMapping.ConsumerCoordinatorNotAvailableCode - else if (status.error == ErrorMapping.NotLeaderForPartitionCode) - ErrorMapping.NotCoordinatorForConsumerCode - else if (status.error == ErrorMapping.MessageSizeTooLargeCode - || status.error == ErrorMapping.MessageSetSizeTooLargeCode - || status.error == ErrorMapping.InvalidFetchSizeCode) - Errors.INVALID_COMMIT_OFFSET_SIZE.code - else - status.error - } - - - // compute the final error codes for the commit response - val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) - (topicAndPartition, responseCode) - else - (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) - } - - // finally trigger the callback logic passed from the API layer - responseCallback(commitStatus) - } - - // call replica manager to append the offset messages - replicaManager.appendMessages( - config.offsetCommitTimeoutMs.toLong, - config.offsetCommitRequiredAcks, - true, // allow appending to internal offset topic - offsetsAndMetadataMessageSet, - putCacheCallback) - } - - /** - * 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.NotCoordinatorForGroup) - }.toMap - } - } - } - - /** - * Asynchronously read the partition from the offsets topic and populate the cache - */ - def loadOffsetsFromLog(offsetsPartition: Int) { - - val topicPartition = TopicAndPartition(GroupCoordinator.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 - cleanupOrLoadMutex synchronized { - while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { - buffer.clear() - val messages = log.read(currOffset, config.loadBufferSize).messageSet.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 { - // special handling for version 0: - // set the expiration time stamp as commit time stamp + server default retention time - val value = OffsetManager.readMessageValue(msgAndOffset.message.payload) - putOffset(key, value.copy ( - expireTimestamp = { - if (value.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - value.commitTimestamp + config.offsetsRetentionMs - else - value.expireTimestamp - } - )) - 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(GroupCoordinator.OffsetsTopicName, partitionId) - - val hw = partitionOpt.map { partition => - partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) - }.getOrElse(-1L) - - hw - } - - 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 removeOffsetsFromCacheForPartition(offsetsPartition: Int) { - var numRemoved = 0 - followerTransitionLock synchronized { - offsetsCache.keys.foreach { key => - if (partitionFor(key.group) == offsetsPartition) { - offsetsCache.remove(key) - numRemoved += 1 - } - } - } - - if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." - .format(numRemoved, TopicAndPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition))) - } - - def shutdown() { - shuttingDown.set(true) - } - - /** - * Gets the partition count of the offsets topic from ZooKeeper. - * If the topic does not exist, the configured partition count is returned. - */ - private def getOffsetsTopicPartitionCount = { - val topic = GroupCoordinator.OffsetsTopicName - val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic)) - if (topicData(topic).nonEmpty) - topicData(topic).size - else - config.offsetsTopicNumPartitions - } -} - -object OffsetManager { - - private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) - - private val CURRENT_OFFSET_SCHEMA_VERSION = 1.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 OFFSET_COMMIT_VALUE_SCHEMA_V1 = new Schema(new Field("offset", INT64), - new Field("metadata", STRING, "Associated metadata.", ""), - new Field("commit_timestamp", INT64), - new Field("expire_timestamp", INT64)) - - private val VALUE_OFFSET_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") - private val VALUE_METADATA_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") - private val VALUE_TIMESTAMP_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") - - private val VALUE_OFFSET_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("offset") - private val VALUE_METADATA_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("metadata") - private val VALUE_COMMIT_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") - private val VALUE_EXPIRE_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") - - // map of versions to schemas - private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), - 1 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V1)) - - 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 - */ - private 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 - */ - private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { - // generate commit value with schema version 1 - val value = new Struct(CURRENT_SCHEMA.valueSchema) - value.set(VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) - value.set(VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) - value.set(VALUE_COMMIT_TIMESTAMP_FIELD_V1, offsetAndMetadata.commitTimestamp) - value.set(VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.expireTimestamp) - 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 - */ - private 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 - */ - private def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { - val structAndVersion = readMessageValueStruct(buffer) - - if (structAndVersion.value == null) { // tombstone - null - } else { - if (structAndVersion.version == 0) { - val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] - val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V0).asInstanceOf[String] - val timestamp = structAndVersion.value.get(VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] - - OffsetAndMetadata(offset, metadata, timestamp) - } else if (structAndVersion.version == 1) { - val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] - val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V1).asInstanceOf[String] - val commitTimestamp = structAndVersion.value.get(VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] - val expireTimestamp = structAndVersion.value.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] - - OffsetAndMetadata(offset, metadata, commitTimestamp, expireTimestamp) - } else { - throw new IllegalStateException("Unknown offset message version") - } - } - } - - private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { - if(buffer == null) { // tombstone - MessageValueStructAndVersion(null, -1) - } else { - val version = buffer.getShort() - val valueSchema = schemaFor(version).valueSchema - val value = valueSchema.read(buffer).asInstanceOf[Struct] - - MessageValueStructAndVersion(value, version) - } - } - - // 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.readMessageValueStruct(ByteBuffer.wrap(value)).value.toString - output.write(formattedKey.getBytes) - output.write("::".getBytes) - output.write(formattedValue.getBytes) - output.write("\n".getBytes) - } - } - -} - -case class MessageValueStructAndVersion(value: Struct, version: Short) - -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/package.html b/core/src/main/scala/kafka/server/package.html deleted file mode 100644 index 56ab9d41a8c9f..0000000000000 --- a/core/src/main/scala/kafka/server/package.html +++ /dev/null @@ -1,19 +0,0 @@ - -The kafka server. \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index db4295c49af73..9487c77d9fb79 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -193,7 +193,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { // get metadata for the topic var parts: Seq[PartitionInfo] = null while (parts == null) - parts = consumer0.partitionsFor(GroupCoordinator.OffsetsTopicName).asScala + parts = consumer0.partitionsFor(GroupCoordinator.GroupMetadataTopicName).asScala assertEquals(1, parts.size) assertNotNull(parts(0).leader()) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index bbc9a544fb3f7..b7ecc9d63232f 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -69,7 +69,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { } // create the consumer offset topic - TestUtils.createTopic(zkUtils, GroupCoordinator.OffsetsTopicName, + TestUtils.createTopic(zkUtils, GroupCoordinator.GroupMetadataTopicName, serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt, servers, diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index 820a825288ab4..0df14c597ac55 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -85,12 +85,12 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging { // create the offset topic val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString, "--replication-factor", "1", - "--topic", GroupCoordinator.OffsetsTopicName)) + "--topic", GroupCoordinator.GroupMetadataTopicName)) TopicCommand.createTopic(zkUtils, createOffsetTopicOpts) // try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't - val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", GroupCoordinator.OffsetsTopicName)) - val deleteOffsetTopicPath = getDeleteTopicPath(GroupCoordinator.OffsetsTopicName) + val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", GroupCoordinator.GroupMetadataTopicName)) + val deleteOffsetTopicPath = getDeleteTopicPath(GroupCoordinator.GroupMetadataTopicName) assertFalse("Delete path for topic shouldn't exist before deletion.", zkUtils.zkClient.exists(deleteOffsetTopicPath)) intercept[AdminOperationException] { TopicCommand.deleteTopic(zkUtils, deleteOffsetTopicOpts) diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 24fba45aa02b6..1e8d04eff1b1b 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -21,7 +21,6 @@ package kafka.consumer import org.junit.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.Test -import kafka.server.OffsetManager import kafka.coordinator.GroupCoordinator @@ -38,8 +37,8 @@ class TopicFilterTest extends JUnitSuite { val topicFilter2 = new Whitelist(".+") assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true)) - assertFalse(topicFilter2.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter2.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter2.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter2.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = false)) val topicFilter3 = new Whitelist("white_listed-topic.+") assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true)) @@ -58,8 +57,8 @@ class TopicFilterTest extends JUnitSuite { assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true)) assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false)) - assertFalse(topicFilter1.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = true)) - assertTrue(topicFilter1.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = false)) + assertFalse(topicFilter1.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = true)) + assertTrue(topicFilter1.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = false)) } @Test From ded31caefeafc9f5a2ef4c38824e4761bad5c158 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 15:10:50 -0700 Subject: [PATCH 07/23] add back package.html --- core/src/main/scala/kafka/server/package.html | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 core/src/main/scala/kafka/server/package.html diff --git a/core/src/main/scala/kafka/server/package.html b/core/src/main/scala/kafka/server/package.html new file mode 100644 index 0000000000000..56ab9d41a8c9f --- /dev/null +++ b/core/src/main/scala/kafka/server/package.html @@ -0,0 +1,19 @@ + +The kafka server. \ No newline at end of file From ebfebd1db57a61b30eb3e296aa3eb59360183860 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 16:28:32 -0700 Subject: [PATCH 08/23] persist synced assignment and client side error handling --- .../internals/AbstractCoordinator.java | 9 ++- .../internals/ConsumerCoordinator.java | 9 ++- .../apache/kafka/common/protocol/Errors.java | 4 +- .../common/requests/HeartbeatResponse.java | 1 + .../common/requests/JoinGroupResponse.java | 1 + .../common/requests/LeaveGroupResponse.java | 1 + .../common/requests/OffsetCommitResponse.java | 1 + .../common/requests/OffsetFetchResponse.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 2 +- .../kafka/common/OffsetMetadataAndError.scala | 2 +- .../coordinator/CoordinatorMetadata.scala | 81 ------------------- .../kafka/coordinator/GroupCoordinator.scala | 44 ++++++---- .../main/scala/kafka/server/KafkaConfig.scala | 19 ++--- .../coordinator/CoordinatorMetadataTest.scala | 71 ---------------- 14 files changed, 60 insertions(+), 187 deletions(-) delete mode 100644 core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala delete mode 100644 core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 4b2a824343485..de009dcbc8933 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -351,6 +351,10 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut } else { onJoinFollower().chain(future); } + } else if (errorCode == Errors.GROUP_LOAD_IN_PROGRESS.code()) { + log.debug("Attempt to join group {} rejected since coordinator is loading the group.", groupId); + // backoff and retry + future.raise(Errors.forCode(errorCode)); } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code()) { // reset the member id and retry immediately AbstractCoordinator.this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; @@ -362,7 +366,7 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut // re-discover the coordinator and retry with backoff coordinatorDead(); log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", - groupId); + groupId); future.raise(Errors.forCode(errorCode)); } else if (errorCode == Errors.INCONSISTENT_GROUP_PROTOCOL.code() || errorCode == Errors.INVALID_SESSION_TIMEOUT.code() @@ -538,6 +542,9 @@ public void handle(HeartbeatResponse heartbeatResponse, RequestFuture futu if (error == Errors.NONE.code()) { log.debug("Received successful heartbeat response."); future.complete(null); + } else if (error == Errors.GROUP_LOAD_IN_PROGRESS.code()) { + log.debug("Attempt to heart beat rejected since coordinator is loading the group, just treat it as successful."); + future.complete(null); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() || error == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 641939abf47d4..97d25c33a618d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -451,7 +451,10 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu // update the local cache only if the partition is still assigned subscriptions.committed(tp, offsetAndMetadata); } else { - if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() + if (errorCode == Errors.GROUP_LOAD_IN_PROGRESS.code()) { + // just retry + future.raise(Errors.GROUP_LOAD_IN_PROGRESS); + } else if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() || errorCode == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { coordinatorDead(); } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code() @@ -511,9 +514,9 @@ public void handle(OffsetFetchResponse response, RequestFuture groupConfig.groupMaxSessionTimeoutMs) { responseCallback(joinError(memberId, Errors.INVALID_SESSION_TIMEOUT.code)) @@ -126,12 +125,12 @@ class GroupCoordinator(val brokerId: Int, // only try to create the group if the group is not unknown AND // the member id is UNKNOWN, if member is specified but group does not // exist we should reject the request - var group = coordinatorMetadata.getGroup(groupId) + var group = groupManager.getGroup(groupId) if (group == null) { if (memberId != JoinGroupRequest.UNKNOWN_MEMBER_ID) { responseCallback(joinError(memberId, Errors.UNKNOWN_MEMBER_ID.code)) } else { - group = coordinatorMetadata.addGroup(groupId, protocolType) + group = groupManager.addGroup(groupId, protocolType) doJoinGroup(group, memberId, sessionTimeoutMs, protocolType, protocols, responseCallback) } } else { @@ -238,7 +237,7 @@ class GroupCoordinator(val brokerId: Int, } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Array.empty, Errors.NOT_COORDINATOR_FOR_GROUP.code) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) responseCallback(Array.empty, Errors.UNKNOWN_MEMBER_ID.code) else @@ -272,7 +271,9 @@ class GroupCoordinator(val brokerId: Int, // propagate the assignment to any awaiting members if (memberId == group.leaderId) { group.transitionTo(Stable) - propagateAssignment(group, groupAssignment) + + // persist the group metadata and upon finish propagate the assignment + groupManager.storeGroup(group, groupAssignment, propagateAssignment) } case Stable => @@ -290,8 +291,10 @@ class GroupCoordinator(val brokerId: Int, responseCallback(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code) } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code) + } else if (isCoordinatorLoadingInProgress(groupId)) { + responseCallback(Errors.GROUP_LOAD_IN_PROGRESS.code) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) { // if the group is marked as dead, it means some other thread has just removed the group // from the coordinator metadata; this is likely that the group has migrated to some other @@ -323,8 +326,10 @@ class GroupCoordinator(val brokerId: Int, responseCallback(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code) } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code) + } else if (isCoordinatorLoadingInProgress(groupId)) { + responseCallback(Errors.GROUP_LOAD_IN_PROGRESS.code) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) { // if the group is marked as dead, it means some other thread has just removed the group // from the coordinator metadata; this is likely that the group has migrated to some other @@ -360,8 +365,10 @@ class GroupCoordinator(val brokerId: Int, responseCallback(offsetMetadata.mapValues(_ => Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code)) } else if (!isCoordinatorForGroup(groupId)) { responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR_FOR_GROUP.code)) + } else if (isCoordinatorLoadingInProgress(groupId)) { + responseCallback(offsetMetadata.mapValues(_ => Errors.GROUP_LOAD_IN_PROGRESS.code)) } else { - val group = coordinatorMetadata.getGroup(groupId) + val group = groupManager.getGroup(groupId) if (group == null) { if (generationId < 0) // the group is not relying on Kafka for partition management, so allow the commit @@ -394,6 +401,8 @@ class GroupCoordinator(val brokerId: Int, partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.GroupCoordinatorNotAvailable)}.toMap } else if (!isCoordinatorForGroup(groupId)) { partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap + } else if (isCoordinatorLoadingInProgress(groupId)) { + partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.GroupLoading)}.toMap } else { // return offsets blindly regardless the current group state since the group may be using // Kafka commit storage without automatic group management @@ -402,12 +411,10 @@ class GroupCoordinator(val brokerId: Int, } def handleGroupImmigration(offsetTopicPartitionId: Int) = { - // TODO we may need to add more logic in KAFKA-2017 groupManager.loadGroupsForPartition(offsetTopicPartitionId) } def handleGroupEmigration(offsetTopicPartitionId: Int) = { - // TODO we may need to add more logic in KAFKA-2017 groupManager.removeGroupsForPartition(offsetTopicPartitionId) } @@ -426,11 +433,12 @@ class GroupCoordinator(val brokerId: Int, } private def propagateAssignment(group: GroupMetadata, - assignment: Map[String, Array[Byte]]) { + assignment: Map[String, Array[Byte]], + errorCode: Short) { for (member <- group.allMembers) { member.assignment = assignment.getOrElse(member.memberId, Array.empty[Byte]) if (member.awaitingSyncCallback != null) { - member.awaitingSyncCallback(member.assignment, Errors.NONE.code) + member.awaitingSyncCallback(member.assignment, errorCode) member.awaitingSyncCallback = null } } @@ -540,7 +548,7 @@ class GroupCoordinator(val brokerId: Int, if (group.isEmpty) { group.transitionTo(Dead) info("Group %s generation %s is dead and removed".format(group.groupId, group.generationId)) - coordinatorMetadata.removeGroup(group.groupId) + groupManager.removeGroup(group.groupId) } } if (!group.is(Dead)) { @@ -592,7 +600,9 @@ class GroupCoordinator(val brokerId: Int, member.awaitingSyncCallback != null || member.latestHeartbeat + member.sessionTimeoutMs > heartbeatDeadline - private def isCoordinatorForGroup(groupId: String) = groupManager.leaderIsLocal(groupManager.partitionFor(groupId)) + private def isCoordinatorForGroup(groupId: String) = groupManager.partitionLeaderIsLocal(groupManager.partitionFor(groupId)) + + private def isCoordinatorLoadingInProgress(groupId: String) = groupManager.partitionLoadingInProgress(groupManager.partitionFor(groupId)) } object GroupCoordinator { diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index b749446bbe7f0..2973dab6b39ed 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -23,6 +23,7 @@ import java.util.Properties import kafka.api.ApiVersion import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig +import kafka.coordinator.OffsetConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs @@ -129,16 +130,16 @@ object Defaults { val ConsumerMaxSessionTimeoutMs = 30000 /** ********* Offset management configuration ***********/ - val OffsetMetadataMaxSize = OffsetManagerConfig.DefaultMaxMetadataSize - val OffsetsLoadBufferSize = OffsetManagerConfig.DefaultLoadBufferSize - val OffsetsTopicReplicationFactor = OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor - val OffsetsTopicPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions - val OffsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes - val OffsetsTopicCompressionCodec: Int = OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec.codec + val OffsetMetadataMaxSize = OffsetConfig.DefaultMaxMetadataSize + val OffsetsLoadBufferSize = OffsetConfig.DefaultLoadBufferSize + val OffsetsTopicReplicationFactor = OffsetConfig.DefaultOffsetsTopicReplicationFactor + val OffsetsTopicPartitions: Int = OffsetConfig.DefaultOffsetsTopicNumPartitions + val OffsetsTopicSegmentBytes: Int = OffsetConfig.DefaultOffsetsTopicSegmentBytes + val OffsetsTopicCompressionCodec: Int = OffsetConfig.DefaultOffsetsTopicCompressionCodec.codec val OffsetsRetentionMinutes: Int = 24 * 60 - val OffsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs - val OffsetCommitTimeoutMs = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs - val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks + val OffsetsRetentionCheckIntervalMs: Long = OffsetConfig.DefaultOffsetsRetentionCheckIntervalMs + val OffsetCommitTimeoutMs = OffsetConfig.DefaultOffsetCommitTimeoutMs + val OffsetCommitRequiredAcks = OffsetConfig.DefaultOffsetCommitRequiredAcks /** ********* Quota Configuration ***********/ val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala deleted file mode 100644 index 49a237bdbee74..0000000000000 --- a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala +++ /dev/null @@ -1,71 +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.coordinator - -import kafka.server.KafkaConfig -import kafka.utils.TestUtils - -import org.junit.Assert._ -import org.junit.{Before, Test} -import org.scalatest.junit.JUnitSuite - -/** - * Test coordinator group and topic metadata management - */ -class CoordinatorMetadataTest extends JUnitSuite { - val DefaultNumPartitions = 8 - val DefaultNumReplicas = 2 - var coordinatorMetadata: CoordinatorMetadata = null - - @Before - def setUp() { - val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") - coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId) - } - - @Test - def testGetNonexistentGroup() { - assertNull(coordinatorMetadata.getGroup("group")) - } - - @Test - def testGetGroup() { - val groupId = "group" - val protocolType = "consumer" - val expected = coordinatorMetadata.addGroup(groupId, protocolType) - val actual = coordinatorMetadata.getGroup(groupId) - assertEquals(expected, actual) - } - - @Test - def testAddGroupReturnsPreexistingGroupIfItAlreadyExists() { - val groupId = "group" - val protocolType = "consumer" - val group1 = coordinatorMetadata.addGroup(groupId, protocolType) - val group2 = coordinatorMetadata.addGroup(groupId, protocolType) - assertEquals(group1, group2) - } - - @Test(expected = classOf[IllegalArgumentException]) - def testRemoveNonexistentGroup() { - val groupId = "group" - val topics = Set("a") - coordinatorMetadata.removeGroup(groupId) - } - -} From e62981f3667165951faa416b928f3d7097547281 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 16:30:25 -0700 Subject: [PATCH 09/23] Add renamed files --- .../coordinator/GroupMetadataManager.scala | 842 ++++++++++++++++++ .../kafka/coordinator/OffsetConfig.scala | 61 ++ 2 files changed, 903 insertions(+) create mode 100644 core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala create mode 100644 core/src/main/scala/kafka/coordinator/OffsetConfig.scala diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala new file mode 100644 index 0000000000000..2b9564ee56c28 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -0,0 +1,842 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.coordinator + +import java.util.concurrent.locks.ReentrantReadWriteLock + +import kafka.utils.CoreUtils._ +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.types.{ArrayOf, 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 org.apache.kafka.common.protocol.types.Type.BYTES +import org.apache.kafka.common.utils.Utils + +import kafka.utils._ +import kafka.common._ +import kafka.message._ +import kafka.log.FileMessageSet +import kafka.metrics.KafkaMetricsGroup +import kafka.common.TopicAndPartition +import kafka.tools.MessageFormatter +import kafka.api.ProducerResponseStatus +import kafka.server.ReplicaManager + +import scala.collection._ +import java.io.PrintStream +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.TimeUnit + +import com.yammer.metrics.core.Gauge + +class GroupMetadataManager(val config: OffsetConfig, + replicaManager: ReplicaManager, + zkUtils: ZkUtils, + scheduler: Scheduler) extends Logging with KafkaMetricsGroup { + + /* offsets cache */ + private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata] + + /* group metadata cache */ + private val groupsCache = new Pool[String, GroupMetadata] + + /* partitions of consumer groups that are being loaded */ + private val loadingPartitions: mutable.Set[Int] = mutable.Set() + + /* lock for expiring stale offsets */ + private val offsetExpireLock = new ReentrantReadWriteLock() + + /* lock for removing offsets of a range partition */ + private val offsetRemoveLock = new ReentrantReadWriteLock() + + /* shutting down flag */ + private val shuttingDown = new AtomicBoolean(false) + + /* number of partitions for the consumer metadata topic */ + private val groupMetadataTopicPartitionCount = getOffsetsTopicPartitionCount + + this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " + + scheduler.schedule(name = "delete-expired-consumer-offsets", + fun = deleteExpiredOffsets, + 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 + } + ) + + def partitionFor(group: String): Int = Utils.abs(group.hashCode) % groupMetadataTopicPartitionCount + + def partitionLeaderIsLocal(partition: Int): Boolean = getHighWatermark(partition) != -1L + + def partitionLoadingInProgress(partition: Int): Boolean = loadingPartitions synchronized loadingPartitions.contains(partition) + + /** + * Get the group associated with the given groupId, or null if not found + */ + def getGroup(groupId: String) = { + groupsCache.get(groupId) + } + + /** + * Add a group or get the group associated with the given groupId if it already exists + */ + def addGroup(groupId: String, protocolType: String): GroupMetadata = { + addGroup(groupId, new GroupMetadata(groupId, protocolType)) + } + + private def addGroup(groupId: String, group: GroupMetadata): GroupMetadata = { + groupsCache.putIfNotExists(groupId, group) + } + + /** + * Remove all metadata associated with the group, including its topics + * @param groupId the groupId of the group we are removing + */ + def removeGroup(groupId: String) { + if (!groupsCache.contains(groupId)) + throw new IllegalArgumentException("Cannot remove non-existing group") + groupsCache.remove(groupId) + } + + def storeGroup(group: GroupMetadata, + groupAssignment: Map[String, Array[Byte]], + responseCallback: (GroupMetadata, Map[String, Array[Byte]], Short) => Unit) { + // construct the message to append + val message = new Message( + key = GroupMetadataManager.groupMetadataKey(group.groupId), + bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment) + ) + + val groupMetadataPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(group.groupId)) + + val groupMetadataMessageSet = Map(groupMetadataPartition -> + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message)) + + // set the callback function to insert offsets into cache after log append completed + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + // the append response should only contain the topics partition + if (responseStatus.size != 1 || ! responseStatus.contains(groupMetadataPartition)) + throw new IllegalStateException("Append status %s should only have one partition %s" + .format(responseStatus, groupMetadataPartition)) + + // construct the commit response status and insert + // the offset and metadata to cache if the append status has no error + val status = responseStatus(groupMetadataPartition) + + val responseCode = + if (status.error != ErrorMapping.NoError) { + debug("Metadata from group %s with generation %d failed when appending to log due to %s" + .format(group.groupId, group.generationId, ErrorMapping.exceptionNameFor(status.error))) + + // transform the log append error code to the corresponding the commit status error code + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code + else + status.error + } else { + Errors.NONE.code + } + + // finally trigger the callback logic passed from the API layer + responseCallback(group, groupAssignment, responseCode) + } + + // call replica manager to append the offset messages + replicaManager.appendMessages( + config.offsetCommitTimeoutMs.toLong, + config.offsetCommitRequiredAcks, + true, // allow appending to internal offset topic + groupMetadataMessageSet, + putCacheCallback) + } + + /** + * Store offsets by appending it to the replicated log and then inserting to cache + */ + def storeOffsets(groupId: String, + consumerId: String, + generationId: Int, + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + // first filter out partitions with offset metadata size exceeding limit + val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => + validateOffsetMetadataLength(offsetAndMetadata.metadata) + } + + // construct the message set to append + val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + new Message( + key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), + bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata) + ) + }.toSeq + + val offsetTopicPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(groupId)) + + val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) + + // set the callback function to insert offsets into cache after log append completed + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + // the append response should only contain the topics partition + if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) + throw new IllegalStateException("Append status %s should only have one partition %s" + .format(responseStatus, offsetTopicPartition)) + + // construct the commit response status and insert + // the offset and metadata to cache if the append status has no error + val status = responseStatus(offsetTopicPartition) + + val responseCode = + if (status.error == ErrorMapping.NoError) { + filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => + putOffset(GroupTopicPartition(groupId, topicAndPartition), offsetAndMetadata) + } + ErrorMapping.NoError + } else { + debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" + .format(filteredOffsetMetadata, groupId, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) + + // transform the log append error code to the corresponding the commit status error code + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code + else + status.error + } + + + // compute the final error codes for the commit response + val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) + (topicAndPartition, responseCode) + else + (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) + } + + // finally trigger the callback logic passed from the API layer + responseCallback(commitStatus) + } + + // call replica manager to append the offset messages + replicaManager.appendMessages( + config.offsetCommitTimeoutMs.toLong, + config.offsetCommitRequiredAcks, + true, // allow appending to internal offset topic + offsetsAndMetadataMessageSet, + putCacheCallback) + } + + /** + * 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) + + /** + * we need to put the leader-is-local check inside the offsetLock to protects against fetching from an empty/cleared + * offset cache (i.e., cleared due to a leader->follower transition right after the check and clear the cache). + */ + inReadLock(offsetRemoveLock) { + if (partitionLeaderIsLocal(offsetsPartition)) { + if (topicPartitions.isEmpty) { + // 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.NotCoordinatorForGroup) + }.toMap + } + } + } + + /** + * Asynchronously read the partition from the offsets topic and populate the cache + */ + def loadGroupsForPartition(offsetsPartition: Int) { + + val topicPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, 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, loadGroupsAndOffsets) + } + } + + def loadGroupsAndOffsets() { + 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 + inWriteLock(offsetExpireLock) { + while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { + buffer.clear() + val messages = log.read(currOffset, config.loadBufferSize).messageSet.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 baseKey = GroupMetadataManager.readMessageKey(msgAndOffset.message.key) + + if (baseKey.isInstanceOf[OffsetKey]) { + // load offset + val key = baseKey.key.asInstanceOf[GroupTopicPartition] + 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 { + // special handling for version 0: + // set the expiration time stamp as commit time stamp + server default retention time + val value = GroupMetadataManager.readOffsetMessageValue(msgAndOffset.message.payload) + putOffset(key, value.copy ( + expireTimestamp = { + if (value.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + value.commitTimestamp + config.offsetsRetentionMs + else + value.expireTimestamp + } + )) + trace("Loaded offset %s for %s.".format(value, key)) + } + } else { + // load group metadata + val groupId = baseKey.key.asInstanceOf[String] + val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, msgAndOffset.message.payload) + + addGroup(groupId, groupMetadata) + } + + 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) + } + } + } + + /** + * 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 removeGroupsForPartition(offsetsPartition: Int) { + var numRemoved = 0 + inWriteLock(offsetRemoveLock) { + offsetsCache.keys.foreach { key => + if (partitionFor(key.group) == offsetsPartition) { + offsetsCache.remove(key) + numRemoved += 1 + } + } + + // TODO: we may also remove the group metadata cache here + } + + if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." + .format(numRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) + } + + /** + * 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) + } + + private def deleteExpiredOffsets() { + debug("Collecting expired offsets.") + val startMs = SystemTime.milliseconds + + val numExpiredOffsetsRemoved = inWriteLock(offsetExpireLock) { + val expiredOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => + offsetAndMetadata.expireTimestamp < startMs + } + + debug("Found %d expired offsets.".format(expiredOffsets.size)) + + // delete the expired offsets from the table and generate tombstone messages to remove them from the log + val tombstonesForPartition = expiredOffsets.map { case (groupTopicAndPartition, offsetAndMetadata) => + val offsetsPartition = partitionFor(groupTopicAndPartition.group) + trace("Removing expired offset and metadata for %s: %s".format(groupTopicAndPartition, offsetAndMetadata)) + + offsetsCache.remove(groupTopicAndPartition) + + val commitKey = GroupMetadataManager.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 expired offsets during their own purge cycles. + tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) => + val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition) + partitionOpt.map { partition => + val appendPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition) + val messages = tombstones.map(_._2).toSeq + + trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) + + try { + // do not need to require acks since even if the tombsone is lost, + // it will be appended again in the next purge cycle + partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages: _*)) + tombstones.size + } + catch { + case t: Throwable => + error("Failed to mark %d expired offsets for deletion in %s.".format(messages.size, appendPartition), t) + // ignore and continue + 0 + } + } + }.sum + } + + info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, SystemTime.milliseconds - startMs)) + } + + private def getHighWatermark(partitionId: Int): Long = { + val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, partitionId) + + val hw = partitionOpt.map { partition => + partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L) + }.getOrElse(-1L) + + hw + } + + /* + * Check if the offset metadata length is valid + */ + private def validateOffsetMetadataLength(metadata: String) : Boolean = { + metadata == null || metadata.length() <= config.maxMetadataSize + } + + def shutdown() { + shuttingDown.set(true) + + // TODO: clear the caches + } + + /** + * Gets the partition count of the offsets topic from ZooKeeper. + * If the topic does not exist, the configured partition count is returned. + */ + private def getOffsetsTopicPartitionCount = { + val topic = GroupCoordinator.GroupMetadataTopicName + val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic)) + if (topicData(topic).nonEmpty) + topicData(topic).size + else + config.offsetsTopicNumPartitions + } +} + + +object GroupMetadataManager { + + private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) + + private val CURRENT_OFFSET_SCHEMA_VERSION = 1.toShort + + private val CURRENT_GROUP_SCHEMA_VERSION = 2.toShort + + private val OFFSET_COMMIT_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING), + new Field("topic", STRING), + new Field("partition", INT32)) + private val OFFSET_KEY_GROUP_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("group") + private val OFFSET_KEY_TOPIC_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("topic") + private val OFFSET_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 OFFSET_VALUE_OFFSET_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") + private val OFFSET_VALUE_METADATA_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") + private val OFFSET_VALUE_TIMESTAMP_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") + + + private val OFFSET_COMMIT_VALUE_SCHEMA_V1 = new Schema(new Field("offset", INT64), + new Field("metadata", STRING, "Associated metadata.", ""), + new Field("commit_timestamp", INT64), + new Field("expire_timestamp", INT64)) + private val OFFSET_VALUE_OFFSET_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("offset") + private val OFFSET_VALUE_METADATA_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("metadata") + private val OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") + private val OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") + + private val GROUP_METADATA_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING)) + private val GROUP_KEY_GROUP_FIELD = GROUP_METADATA_KEY_SCHEMA_V0.get("group") + + private val MEMBER_METADATA_V0 = new Schema(new Field("member_id", STRING), + new Field("session_timeout", INT32), + new Field("subscription", BYTES), + new Field("assignment", BYTES)) + private val MEMBER_METADATA_MEMBER_ID_V0 = MEMBER_METADATA_V0.get("member_id") + private val MEMBER_METADATA_SESSION_TIMEOUT_V0 = MEMBER_METADATA_V0.get("session_timeout") + private val MEMBER_METADATA_SUBSCRIPTION_V0 = MEMBER_METADATA_V0.get("subscription") + private val MEMBER_METADATA_ASSIGNMENT_V0 = MEMBER_METADATA_V0.get("assignment") + + + private val GROUP_METADATA_VALUE_SCHEMA_V0 = new Schema(new Field("protocol_type", STRING), + new Field("generation", INT32), + new Field("protocol", STRING), + new Field("leader", STRING), + new Field("members", new ArrayOf(MEMBER_METADATA_V0))) + private val GROUP_METADATA_PROTOCOL_TYPE_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("protocol_type") + private val GROUP_METADATA_GENERATION_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("generation") + private val GROUP_METADATA_PROTOCOL_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("protocol") + private val GROUP_METADATA_LEADER_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("leader") + private val GROUP_METADATA_MEMBERS_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("members") + + // map of versions to schemas + private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), + 1 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V1), + 2 -> KeyAndValueSchemas(GROUP_METADATA_KEY_SCHEMA_V0, GROUP_METADATA_VALUE_SCHEMA_V0)) + + private val CURRENT_OFFSET_SCHEMA = schemaFor(CURRENT_OFFSET_SCHEMA_VERSION) + private val CURRENT_GROUP_SCHEMA = schemaFor(CURRENT_GROUP_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 + */ + private def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { + val key = new Struct(CURRENT_OFFSET_SCHEMA.keySchema) + key.set(OFFSET_KEY_GROUP_FIELD, group) + key.set(OFFSET_KEY_TOPIC_FIELD, topic) + key.set(OFFSET_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 key for group metadata message for given group + * + * @return key bytes for group metadata message + */ + private def groupMetadataKey(group: String): Array[Byte] = { + val key = new Struct(CURRENT_GROUP_SCHEMA.keySchema) + key.set(GROUP_KEY_GROUP_FIELD, group) + + val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf) + byteBuffer.putShort(CURRENT_GROUP_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 + */ + private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { + // generate commit value with schema version 1 + val value = new Struct(CURRENT_OFFSET_SCHEMA.valueSchema) + value.set(OFFSET_VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) + value.set(OFFSET_VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) + value.set(OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1, offsetAndMetadata.commitTimestamp) + value.set(OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.expireTimestamp) + val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) + byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + value.writeTo(byteBuffer) + byteBuffer.array() + } + + /** + * Generates the payload for group metadata message from given offset and metadata + * assuming the generation id, selected protocol, leader and member assignment are all available + * + * @param groupMetadata + * @return payload for offset commit message + */ + private def groupMetadataValue(groupMetadata: GroupMetadata, assignment: Map[String, Array[Byte]]): Array[Byte] = { + // generate commit value with schema version 1 + val value = new Struct(CURRENT_GROUP_SCHEMA.valueSchema) + value.set(GROUP_METADATA_PROTOCOL_TYPE_V0, groupMetadata.protocolType) + value.set(GROUP_METADATA_GENERATION_V0, groupMetadata.generationId) + value.set(GROUP_METADATA_PROTOCOL_V0, groupMetadata.protocol) + value.set(GROUP_METADATA_LEADER_V0, groupMetadata.leaderId) + + val memberArray = groupMetadata.allMembers.map { + case memberMetadata => + val memberStruct = value.instance(GROUP_METADATA_MEMBERS_V0) + memberStruct.set(MEMBER_METADATA_MEMBER_ID_V0, memberMetadata.memberId) + memberStruct.set(MEMBER_METADATA_SESSION_TIMEOUT_V0, memberMetadata.sessionTimeoutMs) + + val selectedProtocol = memberMetadata.supportedProtocols.filter(_._1.equals(groupMetadata.protocol)) + assert(selectedProtocol.length == 1) + + memberStruct.set(MEMBER_METADATA_SUBSCRIPTION_V0, selectedProtocol.head._2) + + val memberAssignment = assignment(memberMetadata.memberId) + assert(memberAssignment != null) + + memberStruct.set(MEMBER_METADATA_ASSIGNMENT_V0, memberAssignment) + + memberStruct + } + + value.set(GROUP_METADATA_MEMBERS_V0, memberArray.toArray) + + 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 + */ + private def readMessageKey(buffer: ByteBuffer): BaseKey = { + val version = buffer.getShort + val keySchema = schemaFor(version).keySchema + val key = keySchema.read(buffer).asInstanceOf[Struct] + + if (version <= CURRENT_OFFSET_SCHEMA_VERSION) { + // version 0 and 1 refer to offset + val group = key.get(OFFSET_KEY_GROUP_FIELD).asInstanceOf[String] + val topic = key.get(OFFSET_KEY_TOPIC_FIELD).asInstanceOf[String] + val partition = key.get(OFFSET_KEY_PARTITION_FIELD).asInstanceOf[Int] + + OffsetKey(version, GroupTopicPartition(group, TopicAndPartition(topic, partition))) + + } else if (version == CURRENT_GROUP_SCHEMA_VERSION) { + // version 2 refers to offset + val group = key.get(GROUP_KEY_GROUP_FIELD).asInstanceOf[String] + + GroupKey(version, group) + } else { + throw new IllegalStateException("Unknown version " + version + " for group metadata message") + } + } + + /** + * 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 + */ + private def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { + val structAndVersion = readMessageValueStruct(buffer) + + if (structAndVersion.value == null) { // tombstone + null + } else { + if (structAndVersion.version == 0) { + val offset = structAndVersion.value.get(OFFSET_VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] + val metadata = structAndVersion.value.get(OFFSET_VALUE_METADATA_FIELD_V0).asInstanceOf[String] + val timestamp = structAndVersion.value.get(OFFSET_VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, timestamp) + } else if (structAndVersion.version == 1) { + val offset = structAndVersion.value.get(OFFSET_VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] + val metadata = structAndVersion.value.get(OFFSET_VALUE_METADATA_FIELD_V1).asInstanceOf[String] + val commitTimestamp = structAndVersion.value.get(OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + val expireTimestamp = structAndVersion.value.get(OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, commitTimestamp, expireTimestamp) + } else { + throw new IllegalStateException("Unknown offset message version") + } + } + } + + /** + * Decodes the group metadata messages' payload and retrieves its member metadatafrom it + * + * @param buffer input byte-buffer + * @return a group metadata object from the message + */ + private def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = { + val structAndVersion = readMessageValueStruct(buffer) + + if (structAndVersion.value == null) { // tombstone + null + } else { + if (structAndVersion.version == CURRENT_GROUP_SCHEMA_VERSION) { + val protocolType = structAndVersion.value.get(GROUP_METADATA_PROTOCOL_TYPE_V0).asInstanceOf[String] + + val group = new GroupMetadata(groupId, protocolType) + + group.generationId = structAndVersion.value.get(GROUP_METADATA_GENERATION_V0).asInstanceOf[Int] + group.leaderId = structAndVersion.value.get(GROUP_METADATA_LEADER_V0).asInstanceOf[String] + group.protocol = structAndVersion.value.get(GROUP_METADATA_PROTOCOL_V0).asInstanceOf[String] + + structAndVersion.value.getArray(GROUP_METADATA_MEMBERS_V0).foreach { + case memberMetadataObj => + val memberMetadata = memberMetadataObj.asInstanceOf[Struct] + val memberId = memberMetadata.get(MEMBER_METADATA_MEMBER_ID_V0).asInstanceOf[String] + val sessionTimeout = memberMetadata.get(MEMBER_METADATA_SESSION_TIMEOUT_V0).asInstanceOf[Int] + val subscription = memberMetadata.get(MEMBER_METADATA_SUBSCRIPTION_V0).asInstanceOf[Array[Byte]] + + val member = new MemberMetadata(memberId, groupId, sessionTimeout, List((group.protocol, subscription))) + + member.assignment = memberMetadata.get(MEMBER_METADATA_ASSIGNMENT_V0).asInstanceOf[Array[Byte]] + + group.add(memberId, member) + } + + group + } else { + throw new IllegalStateException("Unknown group metadata message version") + } + } + } + + private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { + if(buffer == null) { // tombstone + MessageValueStructAndVersion(null, -1) + } else { + val version = buffer.getShort + val valueSchema = schemaFor(version).valueSchema + val value = valueSchema.read(buffer).asInstanceOf[Struct] + + MessageValueStructAndVersion(value, version) + } + } + + // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. + // (specify --formatter "kafka.coordinator.GroupMetadataManager\$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 GroupMetadataManager.readMessageKey(ByteBuffer.wrap(key)).toString + val formattedValue = if (value == null) "NULL" else GroupMetadataManager.readMessageValueStruct(ByteBuffer.wrap(value)).value.toString + output.write(formattedKey.getBytes) + output.write("::".getBytes) + output.write(formattedValue.getBytes) + output.write("\n".getBytes) + } + } + +} + +case class MessageValueStructAndVersion(value: Struct, version: Short) + +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) +} + +trait BaseKey{ + def version: Short + def key: Object +} + +case class OffsetKey(version: Short, key: GroupTopicPartition) extends BaseKey + +case class GroupKey(version: Short, key: String) extends BaseKey + diff --git a/core/src/main/scala/kafka/coordinator/OffsetConfig.scala b/core/src/main/scala/kafka/coordinator/OffsetConfig.scala new file mode 100644 index 0000000000000..92f56b2076883 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/OffsetConfig.scala @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.coordinator + +import kafka.message.{NoCompressionCodec, CompressionCodec} + +/** + * 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 expired 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 OffsetConfig(maxMetadataSize: Int = OffsetConfig.DefaultMaxMetadataSize, + loadBufferSize: Int = OffsetConfig.DefaultLoadBufferSize, + offsetsRetentionMs: Long = OffsetConfig.DefaultOffsetRetentionMs, + offsetsRetentionCheckIntervalMs: Long = OffsetConfig.DefaultOffsetsRetentionCheckIntervalMs, + offsetsTopicNumPartitions: Int = OffsetConfig.DefaultOffsetsTopicNumPartitions, + offsetsTopicSegmentBytes: Int = OffsetConfig.DefaultOffsetsTopicSegmentBytes, + offsetsTopicReplicationFactor: Short = OffsetConfig.DefaultOffsetsTopicReplicationFactor, + offsetsTopicCompressionCodec: CompressionCodec = OffsetConfig.DefaultOffsetsTopicCompressionCodec, + offsetCommitTimeoutMs: Int = OffsetConfig.DefaultOffsetCommitTimeoutMs, + offsetCommitRequiredAcks: Short = OffsetConfig.DefaultOffsetCommitRequiredAcks) + +object OffsetConfig { + val DefaultMaxMetadataSize = 4096 + val DefaultLoadBufferSize = 5*1024*1024 + val DefaultOffsetRetentionMs = 24*60*60*1000L + val DefaultOffsetsRetentionCheckIntervalMs = 600000L + val DefaultOffsetsTopicNumPartitions = 50 + val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 + val DefaultOffsetsTopicReplicationFactor = 3.toShort + val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec + val DefaultOffsetCommitTimeoutMs = 5000 + val DefaultOffsetCommitRequiredAcks = (-1).toShort +} \ No newline at end of file From 5ae8c914b0583d7ef86aafdf83967b30e408824f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 16:42:30 -0700 Subject: [PATCH 10/23] bindly accepts HB while loading in progress --- .../kafka/clients/consumer/internals/AbstractCoordinator.java | 3 --- .../org/apache/kafka/common/requests/HeartbeatResponse.java | 1 - core/src/main/scala/kafka/coordinator/GroupCoordinator.scala | 3 ++- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index de009dcbc8933..d8f3c25ec907b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -542,9 +542,6 @@ public void handle(HeartbeatResponse heartbeatResponse, RequestFuture futu if (error == Errors.NONE.code()) { log.debug("Received successful heartbeat response."); future.complete(null); - } else if (error == Errors.GROUP_LOAD_IN_PROGRESS.code()) { - log.debug("Attempt to heart beat rejected since coordinator is loading the group, just treat it as successful."); - future.complete(null); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() || error == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index 1345e35425865..48cb4c0f016d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -27,7 +27,6 @@ public class HeartbeatResponse extends AbstractRequestResponse { /** * Possible error code: * - * GROUP_LOAD_IN_PROGRESS (14) * GROUP_COORDINATOR_NOT_AVAILABLE (15) * NOT_COORDINATOR_FOR_GROUP (16) * ILLEGAL_GENERATION (22) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 8c393de9e048c..efde1b709e254 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -327,7 +327,8 @@ class GroupCoordinator(val brokerId: Int, } else if (!isCoordinatorForGroup(groupId)) { responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code) } else if (isCoordinatorLoadingInProgress(groupId)) { - responseCallback(Errors.GROUP_LOAD_IN_PROGRESS.code) + // the group is still loading, so respond just blindly + responseCallback(Errors.NONE.code) } else { val group = groupManager.getGroup(groupId) if (group == null) { From 59b211b78d76f4dae600e6b75ec1080e26365982 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 17:20:14 -0700 Subject: [PATCH 11/23] minor fixes --- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../unit/kafka/admin/TopicCommandTest.scala | 2 +- .../GroupCoordinatorResponseTest.scala | 127 +++++++++--------- 3 files changed, 66 insertions(+), 65 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index b1db12a47325a..3d484b84d3cf2 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -148,7 +148,7 @@ class AuthorizerIntegrationTest extends KafkaServerTestHarness { consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT) // create the consumer offset topic - TestUtils.createTopic(zkUtils, GroupCoordinator.OffsetsTopicName, + TestUtils.createTopic(zkUtils, GroupCoordinator.GroupMetadataTopicName, 1, 1, servers, diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index 0df14c597ac55..f14e25a6d990b 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -88,7 +88,7 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging { "--topic", GroupCoordinator.GroupMetadataTopicName)) TopicCommand.createTopic(zkUtils, createOffsetTopicOpts) - // try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't + // try to delete the GroupCoordinator.GroupMetadataTopicName and make sure it doesn't val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", GroupCoordinator.GroupMetadataTopicName)) val deleteOffsetTopicPath = getDeleteTopicPath(GroupCoordinator.GroupMetadataTopicName) assertFalse("Delete path for topic shouldn't exist before deletion.", zkUtils.zkClient.exists(deleteOffsetTopicPath)) diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 5eaaea8c77a89..c07da7f8fc27a 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -22,7 +22,8 @@ import java.util.concurrent.TimeUnit import org.junit.Assert._ import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import kafka.server.{OffsetManager, KafkaConfig} +import kafka.server.KafkaConfig +import kafka.coordinator.GroupMetadataManager import kafka.utils.TestUtils import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest} @@ -51,21 +52,21 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val ConsumerMaxSessionTimeout = 1000 val DefaultSessionTimeout = 500 var consumerCoordinator: GroupCoordinator = null - var offsetManager : OffsetManager = null + var groupManager : GroupMetadataManager = null @Before def setUp() { val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) - offsetManager = EasyMock.createStrictMock(classOf[OffsetManager]) - consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), null, offsetManager) + groupManager = EasyMock.createStrictMock(classOf[GroupMetadataManager]) + consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), null, groupManager) consumerCoordinator.startup() } @After def tearDown() { - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) consumerCoordinator.shutdown() } @@ -164,7 +165,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, "copycat", List(("range", metadata)), isCoordinatorForGroup = true) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) @@ -183,7 +184,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, List(("roundrobin", metadata)), isCoordinatorForGroup = true) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) @@ -202,7 +203,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, otherJoinGroupResult.errorCode) @@ -240,12 +241,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.memberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, otherMemberId, 1, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult) } @@ -264,7 +265,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @@ -283,12 +284,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) } @@ -308,12 +309,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, heartbeatResult) } @@ -352,12 +353,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val unknownMemberId = "blah" val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId, true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, unknownMemberSyncResult._2) @@ -377,7 +378,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) // send the sync group with an invalid generation val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map.empty, true) assertEquals(Errors.ILLEGAL_GENERATION.code, syncGroupResult._2) @@ -400,15 +401,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true); @@ -424,7 +425,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val nextGenerationId = joinResult.generationId // this shouldn't cause a rebalance since protocol information hasn't changed - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -445,14 +446,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) // join groups from the leader should force the group to rebalance, which allows the // leader to push new assignments when local metadata changes - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val secondJoinResult = joinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -477,15 +478,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true); @@ -502,7 +503,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // with no leader SyncGroup, the follower's request should failure with an error indicating // that it should rejoin - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, isCoordinatorForGroup = true) val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100) @@ -526,15 +527,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true); @@ -553,13 +554,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val followerId = otherJoinResult.memberId val followerAssignment = Array[Byte](1) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaderSyncResult._2) assertEquals(leaderAssignment, leaderSyncResult._1) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, followerSyncResult._2) @@ -583,16 +584,16 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, joinGroupResult.leaderId) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true); @@ -611,10 +612,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, joinResult.leaderId) assertEquals(firstMemberId, otherJoinResult.leaderId) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId, isCoordinatorForGroup = true) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaderSyncResult._2) @@ -665,7 +666,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, Map(tp -> offset), true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, commitOffsetResult(tp)) } @@ -686,12 +687,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, joinGroupErrorCode) // Then join with a new consumer to trigger a rebalance - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @@ -712,7 +713,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(1, initialGenerationId) assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) val nextGenerationId = otherJoinGroupResult.generationId @@ -753,7 +754,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val leaveGroupResult = leaveGroup(groupId, otherMemberId, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult) } @@ -772,7 +773,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(offsetManager) + EasyMock.reset(groupManager) val leaveGroupResult = leaveGroup(groupId, assignedMemberId, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaveGroupResult) } @@ -813,9 +814,9 @@ class GroupCoordinatorResponseTest extends JUnitSuite { protocols: List[(String, Array[Byte])], isCoordinatorForGroup: Boolean): Future[JoinGroupResult] = { val (responseFuture, responseCallback) = setupJoinGroupCallback - EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(offsetManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(groupManager) consumerCoordinator.handleJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, responseCallback) responseFuture } @@ -827,9 +828,9 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assignment: Map[String, Array[Byte]], isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(offsetManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(groupManager) consumerCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) responseFuture } @@ -839,9 +840,9 @@ class GroupCoordinatorResponseTest extends JUnitSuite { memberId: String, isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(offsetManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(groupManager) consumerCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback) responseFuture } @@ -880,9 +881,9 @@ class GroupCoordinatorResponseTest extends JUnitSuite { generationId: Int, isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(offsetManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(groupManager) consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } @@ -897,23 +898,23 @@ class GroupCoordinatorResponseTest extends JUnitSuite { offsets: Map[TopicAndPartition, OffsetAndMetadata], isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback - EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) val storeOffsetAnswer = new IAnswer[Unit] { override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code)) } - EasyMock.expect(offsetManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback)) + EasyMock.expect(groupManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback)) .andAnswer(storeOffsetAnswer) - EasyMock.replay(offsetManager) + EasyMock.replay(groupManager) consumerCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } private def leaveGroup(groupId: String, consumerId: String, isCoordinatorForGroup: Boolean): LeaveGroupCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(offsetManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.replay(groupManager) consumerCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } From d257ede652b246d7be2044f74ae2027f6b3cf6de Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 17:24:14 -0700 Subject: [PATCH 12/23] some more minor fixes on unit tests --- .../coordinator/GroupCoordinatorResponseTest.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index c07da7f8fc27a..3a608902f1a17 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -23,7 +23,6 @@ import java.util.concurrent.TimeUnit import org.junit.Assert._ import kafka.common.{OffsetAndMetadata, TopicAndPartition} import kafka.server.KafkaConfig -import kafka.coordinator.GroupMetadataManager import kafka.utils.TestUtils import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest} @@ -815,7 +814,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): Future[JoinGroupResult] = { val (responseFuture, responseCallback) = setupJoinGroupCallback EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.replay(groupManager) consumerCoordinator.handleJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, responseCallback) responseFuture @@ -829,7 +828,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.replay(groupManager) consumerCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) responseFuture @@ -841,7 +840,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.replay(groupManager) consumerCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback) responseFuture @@ -882,7 +881,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.replay(groupManager) consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) @@ -899,7 +898,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) val storeOffsetAnswer = new IAnswer[Unit] { override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code)) } @@ -913,7 +912,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def leaveGroup(groupId: String, consumerId: String, isCoordinatorForGroup: Boolean): LeaveGroupCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.replay(groupManager) consumerCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) From edd27a61c9bad7168f0975eb35ed077b0660b793 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 18:25:08 -0700 Subject: [PATCH 13/23] fix Pool.put() call to return the group itself --- core/src/main/scala/kafka/common/ErrorMapping.scala | 2 +- core/src/main/scala/kafka/coordinator/GroupCoordinator.scala | 2 +- core/src/main/scala/kafka/coordinator/GroupMetadata.scala | 2 +- .../main/scala/kafka/coordinator/GroupMetadataManager.scala | 3 ++- core/src/main/scala/kafka/server/KafkaApis.scala | 1 + 5 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index 23224eca9370b..81cb51b79e3a0 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -59,7 +59,7 @@ object ErrorMapping { // 26: INVALID_SESSION_TIMEOUT // 27: COMMITTING_PARTITIONS_NOT_ASSIGNED // 28: INVALID_COMMIT_OFFSET_SIZE - val AuthorizationCode: Short = 29; + val AuthorizationCode: Short = 29 private val exceptionToCode = Map[Class[Throwable], Short]( diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index efde1b709e254..752fdd0c54056 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -553,7 +553,7 @@ class GroupCoordinator(val brokerId: Int, } } if (!group.is(Dead)) { - group.initNextGeneration + group.initNextGeneration() info("Stabilized group %s generation %s".format(group.groupId, group.generationId)) // trigger the awaiting join group response callback for all the members after rebalancing diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala index 60ee98751f735..20433d47ce1e1 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala @@ -188,7 +188,7 @@ private[coordinator] class GroupMetadata(val groupId: String, val protocolType: isEmpty || (memberProtocols & candidateProtocols).nonEmpty } - def initNextGeneration = { + def initNextGeneration() = { assert(notYetRejoinedMembers == List.empty[MemberMetadata]) generationId += 1 protocol = selectProtocol diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 2b9564ee56c28..de8abcca232bd 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -112,7 +112,8 @@ class GroupMetadataManager(val config: OffsetConfig, } private def addGroup(groupId: String, group: GroupMetadata): GroupMetadata = { - groupsCache.putIfNotExists(groupId, group) + groupsCache.put(groupId, group) + group } /** diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c234558060ca6..e65bfc73ff169 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -717,6 +717,7 @@ class KafkaApis(val requestChannel: RequestChannel, val members = joinResult.members map { case (memberId, metadataArray) => (memberId, ByteBuffer.wrap(metadataArray)) } val responseBody = new JoinGroupResponse(joinResult.errorCode, joinResult.generationId, joinResult.subProtocol, joinResult.memberId, joinResult.leaderId, members) + trace("Sending join group response %s for correlation id %d to client %s." .format(responseBody, request.header.correlationId, request.header.clientId)) requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) From 4bf66470a56ffcb443c4fac9cbc924771ee47a6d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 18:49:30 -0700 Subject: [PATCH 14/23] convert between byte array and byte buffer --- .../scala/kafka/coordinator/GroupMetadataManager.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index de8abcca232bd..d11f549491821 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -674,12 +674,12 @@ object GroupMetadataManager { val selectedProtocol = memberMetadata.supportedProtocols.filter(_._1.equals(groupMetadata.protocol)) assert(selectedProtocol.length == 1) - memberStruct.set(MEMBER_METADATA_SUBSCRIPTION_V0, selectedProtocol.head._2) + memberStruct.set(MEMBER_METADATA_SUBSCRIPTION_V0, ByteBuffer.wrap(selectedProtocol.head._2)) val memberAssignment = assignment(memberMetadata.memberId) assert(memberAssignment != null) - memberStruct.set(MEMBER_METADATA_ASSIGNMENT_V0, memberAssignment) + memberStruct.set(MEMBER_METADATA_ASSIGNMENT_V0, ByteBuffer.wrap(memberAssignment)) memberStruct } @@ -778,11 +778,11 @@ object GroupMetadataManager { val memberMetadata = memberMetadataObj.asInstanceOf[Struct] val memberId = memberMetadata.get(MEMBER_METADATA_MEMBER_ID_V0).asInstanceOf[String] val sessionTimeout = memberMetadata.get(MEMBER_METADATA_SESSION_TIMEOUT_V0).asInstanceOf[Int] - val subscription = memberMetadata.get(MEMBER_METADATA_SUBSCRIPTION_V0).asInstanceOf[Array[Byte]] + val subscription = Utils.toArray(memberMetadata.get(MEMBER_METADATA_SUBSCRIPTION_V0).asInstanceOf[ByteBuffer]) val member = new MemberMetadata(memberId, groupId, sessionTimeout, List((group.protocol, subscription))) - member.assignment = memberMetadata.get(MEMBER_METADATA_ASSIGNMENT_V0).asInstanceOf[Array[Byte]] + member.assignment = Utils.toArray(memberMetadata.get(MEMBER_METADATA_ASSIGNMENT_V0).asInstanceOf[ByteBuffer]) group.add(memberId, member) } From 49ffb82cf2e987932e97b324b0841ac0ea0b4dc1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 29 Oct 2015 20:06:11 -0700 Subject: [PATCH 15/23] wip fixing CoordinatorResponseTest --- .../kafka/coordinator/GroupCoordinator.scala | 1 - .../GroupCoordinatorResponseTest.scala | 283 ++++++------------ 2 files changed, 93 insertions(+), 191 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 752fdd0c54056..63a69a79ee6f6 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -633,7 +633,6 @@ object GroupCoordinator { } def create(config: KafkaConfig, - zkUtils: ZkUtils, groupManager: GroupMetadataManager): GroupCoordinator = { val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize, loadBufferSize = config.offsetsLoadBufferSize, diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 3a608902f1a17..86728fbb3c80f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -22,14 +22,15 @@ import java.util.concurrent.TimeUnit import org.junit.Assert._ import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.server.{ReplicaManager, KafkaConfig} +import kafka.utils.{KafkaScheduler, ZkUtils, TestUtils} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest} import org.easymock.{IAnswer, EasyMock} import org.junit.{After, Before, Test} import org.scalatest.junit.JUnitSuite +import scala.collection._ import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future, Promise} @@ -51,31 +52,50 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val ConsumerMaxSessionTimeout = 1000 val DefaultSessionTimeout = 500 var consumerCoordinator: GroupCoordinator = null - var groupManager : GroupMetadataManager = null + var replicaManager : ReplicaManager = null + + final val groupId = "groupId" + final val protocolType = "consumer" + final val memberId = "memberId" + final val metadata = Array[Byte]() + final val protocols = List(("range", metadata)) @Before def setUp() { val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) - groupManager = EasyMock.createStrictMock(classOf[GroupMetadataManager]) - consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), null, groupManager) + + replicaManager = EasyMock.createStrictMock(classOf[ReplicaManager]) + EasyMock.expect(replicaManager.config).andReturn(KafkaConfig.fromProps(props)) + EasyMock.replay(replicaManager) + + val zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) + val assignment = new mutable.HashMap[String, Map[Int, Seq[Int]]]() + assignment += (GroupCoordinator.GroupMetadataTopicName -> Map(0 -> Seq(1))) + EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(GroupCoordinator.GroupMetadataTopicName))) + .andReturn(assignment) + EasyMock.replay(zkUtils) + + // create nice mock since we don't particularly care about scheduler calls + val scheduler = EasyMock.createNiceMock(classOf[KafkaScheduler]) + EasyMock.replay(scheduler) + + consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), zkUtils, replicaManager, scheduler) consumerCoordinator.startup() + EasyMock.reset(zkUtils) + EasyMock.reset(replicaManager) } @After def tearDown() { - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) consumerCoordinator.shutdown() } @Test def testJoinGroupWrongCoordinator() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = false) @@ -85,11 +105,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupSessionTimeoutTooSmall() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, ConsumerMinSessionTimeout - 1, protocolType, protocols, isCoordinatorForGroup = true) @@ -99,11 +115,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupSessionTimeoutTooLarge() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, ConsumerMaxSessionTimeout + 1, protocolType, protocols, isCoordinatorForGroup = true) @@ -113,12 +125,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupUnknownConsumerNewGroup() { - val groupId = "groupId" - val memberId = "memberId" - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) val joinGroupErrorCode = joinGroupResult.errorCode @@ -129,9 +135,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testInvalidGroupId() { val groupId = "" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -140,11 +143,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testValidJoinGroup() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -154,17 +153,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupInconsistentProtocolType() { - val groupId = "groupId" - val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, "consumer", List(("range", metadata)), isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, "copycat", List(("range", metadata)), isCoordinatorForGroup = true) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) @@ -172,10 +167,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupInconsistentGroupProtocol() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val protocolType = "consumer" - val metadata = Array[Byte]() val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID @@ -183,7 +175,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, List(("roundrobin", metadata)), isCoordinatorForGroup = true) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) @@ -191,18 +183,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupUnknownConsumerExistingGroup() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "memberId" - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, otherJoinGroupResult.errorCode) @@ -210,53 +198,41 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testHeartbeatWrongCoordinator() { - val groupId = "groupId" - val consumerId = "memberId" - val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = false) + val heartbeatResult = heartbeat(groupId, memberId, -1, isCoordinatorForGroup = false) assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, heartbeatResult) } @Test def testHeartbeatUnknownGroup() { - val groupId = "groupId" - val consumerId = "memberId" - val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = true) + val heartbeatResult = heartbeat(groupId, memberId, -1, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult) } @Test def testHeartbeatUnknownConsumerExistingGroup() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "memberId" - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.memberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val heartbeatResult = heartbeat(groupId, otherMemberId, 1, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult) } @Test def testHeartbeatRebalanceInProgress() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -264,18 +240,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @Test def testHeartbeatIllegalGeneration() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -283,23 +255,19 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) } @Test def testValidHeartbeat() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -308,20 +276,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, heartbeatResult) } @Test def testSyncGroupNotCoordinator() { - val groupId = "groupId" - val memberId = "member" val generation = 1 val syncGroupResult = syncGroupFollower(groupId, generation, memberId, false) @@ -330,8 +296,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testSyncGroupFromUnknownGroup() { - val groupId = "groupId" - val memberId = "member" val generation = 1 val syncGroupResult = syncGroupFollower(groupId, generation, memberId, true) @@ -340,11 +304,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testSyncGroupFromUnknownMember() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -352,12 +312,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val unknownMemberId = "blah" val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId, true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, unknownMemberSyncResult._2) @@ -365,11 +325,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testSyncGroupFromIllegalGeneration() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -377,7 +333,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) // send the sync group with an invalid generation val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map.empty, true) assertEquals(Errors.ILLEGAL_GENERATION.code, syncGroupResult._2) @@ -385,10 +341,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupFromUnchangedFollowerDoesNotRebalance() { - val groupId = "groupId" - val protocolType = "consumer" - val protocols = List(("range", Array[Byte]())) - // to get a group of two members: // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member @@ -400,17 +352,17 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true); + isCoordinatorForGroup = true) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -424,7 +376,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val nextGenerationId = joinResult.generationId // this shouldn't cause a rebalance since protocol information hasn't changed - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -434,10 +386,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupFromUnchangedLeaderShouldRebalance() { - val groupId = "groupId" - val protocolType = "consumer" - val protocols = List(("range", Array[Byte]())) - val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) val firstMemberId = firstJoinResult.memberId @@ -445,14 +393,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) // join groups from the leader should force the group to rebalance, which allows the // leader to push new assignments when local metadata changes - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val secondJoinResult = joinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -462,10 +410,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testLeaderFailureInSyncGroup() { - val groupId = "groupId" - val protocolType = "consumer" - val protocols = List(("range", Array[Byte]())) - // to get a group of two members: // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member @@ -477,17 +421,17 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true); + isCoordinatorForGroup = true) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -502,7 +446,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // with no leader SyncGroup, the follower's request should failure with an error indicating // that it should rejoin - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, isCoordinatorForGroup = true) val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100) @@ -511,10 +455,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testSyncGroupFollowerAfterLeader() { - val groupId = "groupId" - val protocolType = "consumer" - val protocols = List(("range", Array[Byte]())) - // to get a group of two members: // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member @@ -526,17 +466,17 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true); + isCoordinatorForGroup = true) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -553,13 +493,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val followerId = otherJoinResult.memberId val followerAssignment = Array[Byte](1) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaderSyncResult._2) assertEquals(leaderAssignment, leaderSyncResult._1) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, followerSyncResult._2) @@ -568,10 +508,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testSyncGroupLeaderAfterFollower() { - val groupId = "groupId" - val protocolType = "consumer" - val protocols = List(("range", Array[Byte]())) - // to get a group of two members: // 1. join and sync with a single member (because we can't immediately join with two members) // 2. join and sync with the first member and a new member @@ -583,18 +519,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, joinGroupResult.leaderId) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true); + isCoordinatorForGroup = true) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -611,10 +547,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, joinResult.leaderId) assertEquals(firstMemberId, otherJoinResult.leaderId) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId, isCoordinatorForGroup = true) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaderSyncResult._2) @@ -627,34 +563,27 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testCommitOffsetFromUnknownGroup() { - val groupId = "groupId" - val consumerId = "consumer" val generationId = 1 val tp = new TopicAndPartition("topic", 0) val offset = OffsetAndMetadata(0) - val commitOffsetResult = commitOffsets(groupId, consumerId, generationId, Map(tp -> offset), true) + val commitOffsetResult = commitOffsets(groupId, memberId, generationId, immutable.Map(tp -> offset), true) assertEquals(Errors.ILLEGAL_GENERATION.code, commitOffsetResult(tp)) } @Test def testCommitOffsetWithDefaultGeneration() { - val groupId = "groupId" val tp = new TopicAndPartition("topic", 0) val offset = OffsetAndMetadata(0) val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, - OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset), true) + OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset), true) assertEquals(Errors.NONE.code, commitOffsetResult(tp)) } @Test def testCommitOffsetInAwaitingSync() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val tp = new TopicAndPartition("topic", 0) val offset = OffsetAndMetadata(0) @@ -665,18 +594,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, Map(tp -> offset), true) + EasyMock.reset(replicaManager) + val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, immutable.Map(tp -> offset), true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, commitOffsetResult(tp)) } @Test def testHeartbeatDuringRebalanceCausesRebalanceInProgress() { - val groupId = "groupId" - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) - // First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts) val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -686,24 +610,20 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, joinGroupErrorCode) // Then join with a new consumer to trigger a rebalance - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @Test def testGenerationIdIncrementsOnRebalance() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -712,7 +632,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(1, initialGenerationId) assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) val nextGenerationId = otherJoinGroupResult.generationId @@ -723,7 +643,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testLeaveGroupWrongCoordinator() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val leaveGroupResult = leaveGroup(groupId, memberId, isCoordinatorForGroup = false) @@ -732,8 +651,6 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testLeaveGroupUnknownGroup() { - val groupId = "groupId" - val memberId = "consumerId" val leaveGroupResult = leaveGroup(groupId, memberId, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult) @@ -741,30 +658,22 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testLeaveGroupUnknownConsumerExistingGroup() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "consumerId" - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val leaveGroupResult = leaveGroup(groupId, otherMemberId, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult) } @Test def testValidLeaveGroup() { - val groupId = "groupId" val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val metadata = Array[Byte]() - val protocolType = "consumer" - val protocols = List(("range", metadata)) val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -772,7 +681,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val leaveGroupResult = leaveGroup(groupId, assignedMemberId, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaveGroupResult) } @@ -813,9 +722,9 @@ class GroupCoordinatorResponseTest extends JUnitSuite { protocols: List[(String, Array[Byte])], isCoordinatorForGroup: Boolean): Future[JoinGroupResult] = { val (responseFuture, responseCallback) = setupJoinGroupCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(groupManager) + + EasyMock.expect(replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, 0)).andReturn() + EasyMock.replay(replicaManager) consumerCoordinator.handleJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, responseCallback) responseFuture } @@ -827,9 +736,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assignment: Map[String, Array[Byte]], isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(groupManager) + + EasyMock.replay(replicaManager) consumerCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) responseFuture } @@ -839,9 +747,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { memberId: String, isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(groupManager) + + EasyMock.replay(replicaManager) consumerCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback) responseFuture } @@ -880,9 +787,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { generationId: Int, isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(groupManager) + + EasyMock.replay(replicaManager) consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } @@ -894,26 +800,23 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def commitOffsets(groupId: String, consumerId: String, generationId: Int, - offsets: Map[TopicAndPartition, OffsetAndMetadata], + offsets: immutable.Map[TopicAndPartition, OffsetAndMetadata], isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) + val storeOffsetAnswer = new IAnswer[Unit] { override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code)) } - EasyMock.expect(groupManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback)) - .andAnswer(storeOffsetAnswer) - EasyMock.replay(groupManager) + + EasyMock.replay(replicaManager) consumerCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } private def leaveGroup(groupId: String, consumerId: String, isCoordinatorForGroup: Boolean): LeaveGroupCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.replay(groupManager) + + EasyMock.replay(replicaManager) consumerCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } From 76dd15ee4ec5500f79ead2d46bf25c8d016cef2d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 30 Oct 2015 10:55:12 -0700 Subject: [PATCH 16/23] change storeGroup to return error code upon latch awaiting --- .../kafka/coordinator/GroupCoordinator.scala | 11 +- .../coordinator/GroupMetadataManager.scala | 51 ++--- .../GroupCoordinatorResponseTest.scala | 183 +++++++++++------- 3 files changed, 142 insertions(+), 103 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 63a69a79ee6f6..b111c93c7b41b 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -273,7 +273,10 @@ class GroupCoordinator(val brokerId: Int, group.transitionTo(Stable) // persist the group metadata and upon finish propagate the assignment - groupManager.storeGroup(group, groupAssignment, propagateAssignment) + val responseCode = groupManager.storeGroup(group, groupAssignment) + + // send the assignment back to members + propagateAssignment(group, groupAssignment, responseCode) } case Stable => @@ -433,9 +436,9 @@ class GroupCoordinator(val brokerId: Int, errorCode=errorCode) } - private def propagateAssignment(group: GroupMetadata, - assignment: Map[String, Array[Byte]], - errorCode: Short) { + def propagateAssignment(group: GroupMetadata, + assignment: Map[String, Array[Byte]], + errorCode: Short) { for (member <- group.allMembers) { member.assignment = assignment.getOrElse(member.memberId, Array.empty[Byte]) if (member.awaitingSyncCallback != null) { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index d11f549491821..e5ca8fc5ed76a 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -42,7 +42,7 @@ import scala.collection._ import java.io.PrintStream import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.TimeUnit +import java.util.concurrent.{CountDownLatch, TimeUnit} import com.yammer.metrics.core.Gauge @@ -127,8 +127,7 @@ class GroupMetadataManager(val config: OffsetConfig, } def storeGroup(group: GroupMetadata, - groupAssignment: Map[String, Array[Byte]], - responseCallback: (GroupMetadata, Map[String, Array[Byte]], Short) => Unit) { + groupAssignment: Map[String, Array[Byte]]): Short = { // construct the message to append val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), @@ -140,6 +139,9 @@ class GroupMetadataManager(val config: OffsetConfig, val groupMetadataMessageSet = Map(groupMetadataPartition -> new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message)) + val responseCode = Errors.NONE.code + val responseLatch = new CountDownLatch(1) + // set the callback function to insert offsets into cache after log append completed def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { // the append response should only contain the topics partition @@ -151,28 +153,25 @@ class GroupMetadataManager(val config: OffsetConfig, // the offset and metadata to cache if the append status has no error val status = responseStatus(groupMetadataPartition) - val responseCode = - if (status.error != ErrorMapping.NoError) { - debug("Metadata from group %s with generation %d failed when appending to log due to %s" - .format(group.groupId, group.generationId, ErrorMapping.exceptionNameFor(status.error))) - - // transform the log append error code to the corresponding the commit status error code - if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) - ErrorMapping.ConsumerCoordinatorNotAvailableCode - else if (status.error == ErrorMapping.NotLeaderForPartitionCode) - ErrorMapping.NotCoordinatorForConsumerCode - else if (status.error == ErrorMapping.MessageSizeTooLargeCode - || status.error == ErrorMapping.MessageSetSizeTooLargeCode - || status.error == ErrorMapping.InvalidFetchSizeCode) - Errors.INVALID_COMMIT_OFFSET_SIZE.code - else - status.error - } else { - Errors.NONE.code - } + if (status.error != ErrorMapping.NoError) { + debug("Metadata from group %s with generation %d failed when appending to log due to %s" + .format(group.groupId, group.generationId, ErrorMapping.exceptionNameFor(status.error))) + + // transform the log append error code to the corresponding the commit status error code + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else if (status.error == ErrorMapping.MessageSizeTooLargeCode + || status.error == ErrorMapping.MessageSetSizeTooLargeCode + || status.error == ErrorMapping.InvalidFetchSizeCode) + Errors.INVALID_COMMIT_OFFSET_SIZE.code + else + status.error + } - // finally trigger the callback logic passed from the API layer - responseCallback(group, groupAssignment, responseCode) + // notify the storing thread to return + responseLatch.countDown() } // call replica manager to append the offset messages @@ -182,6 +181,10 @@ class GroupMetadataManager(val config: OffsetConfig, true, // allow appending to internal offset topic groupMetadataMessageSet, putCacheCallback) + + responseLatch.await() + + responseCode } /** diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 86728fbb3c80f..4fe0b47d56c96 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -22,15 +22,14 @@ import java.util.concurrent.TimeUnit import org.junit.Assert._ import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import kafka.server.{ReplicaManager, KafkaConfig} -import kafka.utils.{KafkaScheduler, ZkUtils, TestUtils} +import kafka.server.KafkaConfig +import kafka.utils.{Pool, TestUtils} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest} import org.easymock.{IAnswer, EasyMock} import org.junit.{After, Before, Test} import org.scalatest.junit.JUnitSuite -import scala.collection._ import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future, Promise} @@ -52,7 +51,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val ConsumerMaxSessionTimeout = 1000 val DefaultSessionTimeout = 500 var consumerCoordinator: GroupCoordinator = null - var replicaManager : ReplicaManager = null + var groupManager : GroupMetadataManager = null final val groupId = "groupId" final val protocolType = "consumer" @@ -60,36 +59,24 @@ class GroupCoordinatorResponseTest extends JUnitSuite { final val metadata = Array[Byte]() final val protocols = List(("range", metadata)) + private val groupsCache = new Pool[String, GroupMetadata] + @Before def setUp() { val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) - replicaManager = EasyMock.createStrictMock(classOf[ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(KafkaConfig.fromProps(props)) - EasyMock.replay(replicaManager) - - val zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) - val assignment = new mutable.HashMap[String, Map[Int, Seq[Int]]]() - assignment += (GroupCoordinator.GroupMetadataTopicName -> Map(0 -> Seq(1))) - EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(GroupCoordinator.GroupMetadataTopicName))) - .andReturn(assignment) - EasyMock.replay(zkUtils) - - // create nice mock since we don't particularly care about scheduler calls - val scheduler = EasyMock.createNiceMock(classOf[KafkaScheduler]) - EasyMock.replay(scheduler) + groupManager = EasyMock.createStrictMock(classOf[GroupMetadataManager]) - consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), zkUtils, replicaManager, scheduler) + consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), groupManager) consumerCoordinator.startup() - EasyMock.reset(zkUtils) - EasyMock.reset(replicaManager) } @After def tearDown() { - EasyMock.reset(replicaManager) + groupsCache.clear() + EasyMock.reset(groupManager) consumerCoordinator.shutdown() } @@ -153,15 +140,16 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupInconsistentProtocolType() { + val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, "consumer", List(("range", metadata)), - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, + protocols, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, "copycat", - List(("range", metadata)), isCoordinatorForGroup = true) + protocols, isCoordinatorForGroup = true) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) } @@ -175,7 +163,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, List(("roundrobin", metadata)), isCoordinatorForGroup = true) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) @@ -190,7 +178,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, otherJoinGroupResult.errorCode) @@ -220,12 +208,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.memberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, otherMemberId, 1, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult) } @@ -240,7 +228,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @@ -255,12 +243,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) } @@ -276,12 +264,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, heartbeatResult) } @@ -312,12 +300,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val unknownMemberId = "blah" val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId, true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, unknownMemberSyncResult._2) @@ -333,7 +321,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) // send the sync group with an invalid generation val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map.empty, true) assertEquals(Errors.ILLEGAL_GENERATION.code, syncGroupResult._2) @@ -352,15 +340,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -376,7 +364,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val nextGenerationId = joinResult.generationId // this shouldn't cause a rebalance since protocol information hasn't changed - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -393,14 +381,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) // join groups from the leader should force the group to rebalance, which allows the // leader to push new assignments when local metadata changes - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val secondJoinResult = joinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -421,15 +409,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -446,7 +434,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // with no leader SyncGroup, the follower's request should failure with an error indicating // that it should rejoin - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, isCoordinatorForGroup = true) val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100) @@ -466,15 +454,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -493,13 +481,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val followerId = otherJoinResult.memberId val followerAssignment = Array[Byte](1) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaderSyncResult._2) assertEquals(leaderAssignment, leaderSyncResult._1) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, followerSyncResult._2) @@ -519,16 +507,16 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, joinGroupResult.leaderId) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) @@ -547,10 +535,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, joinResult.leaderId) assertEquals(firstMemberId, otherJoinResult.leaderId) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId, isCoordinatorForGroup = true) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaderSyncResult._2) @@ -567,7 +555,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val tp = new TopicAndPartition("topic", 0) val offset = OffsetAndMetadata(0) - val commitOffsetResult = commitOffsets(groupId, memberId, generationId, immutable.Map(tp -> offset), true) + val commitOffsetResult = commitOffsets(groupId, memberId, generationId, Map(tp -> offset), true) assertEquals(Errors.ILLEGAL_GENERATION.code, commitOffsetResult(tp)) } @@ -577,7 +565,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val offset = OffsetAndMetadata(0) val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, - OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset), true) + OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset), true) assertEquals(Errors.NONE.code, commitOffsetResult(tp)) } @@ -594,8 +582,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) - val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, immutable.Map(tp -> offset), true) + EasyMock.reset(groupManager) + val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, Map(tp -> offset), true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, commitOffsetResult(tp)) } @@ -610,12 +598,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, joinGroupErrorCode) // Then join with a new consumer to trigger a rebalance - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @@ -632,7 +620,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(1, initialGenerationId) assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) val nextGenerationId = otherJoinGroupResult.generationId @@ -666,7 +654,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val leaveGroupResult = leaveGroup(groupId, otherMemberId, isCoordinatorForGroup = true) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult) } @@ -681,7 +669,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(replicaManager) + EasyMock.reset(groupManager) val leaveGroupResult = leaveGroup(groupId, assignedMemberId, isCoordinatorForGroup = true) assertEquals(Errors.NONE.code, leaveGroupResult) } @@ -723,8 +711,21 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): Future[JoinGroupResult] = { val (responseFuture, responseCallback) = setupJoinGroupCallback - EasyMock.expect(replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, 0)).andReturn() - EasyMock.replay(replicaManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) + EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + if (!groupsCache.contains(groupId)) { + val groupMetadata = new GroupMetadata(groupId, protocolType) + groupMetadata.add(memberId, new MemberMetadata(memberId, groupId, sessionTimeout, protocols)) + groupsCache.putIfNotExists(groupId, groupMetadata) + + EasyMock.expect(groupManager.addGroup(groupId, protocolType)).andReturn(groupMetadata) + } + + EasyMock.replay(groupManager) + consumerCoordinator.handleJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, responseCallback) responseFuture } @@ -737,7 +738,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - EasyMock.replay(replicaManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + if (groupsCache.contains(groupId)) { + EasyMock.expect(groupManager.storeGroup(groupsCache.get(groupId), assignment)).andReturn(Errors.NONE.code) + } + EasyMock.replay(groupManager) + consumerCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) responseFuture } @@ -748,7 +756,11 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - EasyMock.replay(replicaManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + EasyMock.replay(groupManager) + consumerCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback) responseFuture } @@ -788,7 +800,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.replay(replicaManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) + EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + EasyMock.replay(groupManager) + consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } @@ -800,7 +818,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def commitOffsets(groupId: String, consumerId: String, generationId: Int, - offsets: immutable.Map[TopicAndPartition, OffsetAndMetadata], + offsets: Map[TopicAndPartition, OffsetAndMetadata], isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback @@ -808,7 +826,15 @@ class GroupCoordinatorResponseTest extends JUnitSuite { override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code)) } - EasyMock.replay(replicaManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) + EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + EasyMock.expect(groupManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback)) + .andAnswer(storeOffsetAnswer) + EasyMock.replay(groupManager) + consumerCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } @@ -816,7 +842,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def leaveGroup(groupId: String, consumerId: String, isCoordinatorForGroup: Boolean): LeaveGroupCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.replay(replicaManager) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) + EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) + EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + EasyMock.expect(groupManager.removeGroup(groupId)) + EasyMock.replay(groupManager) + consumerCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } From e03cfbd731121477504881a1e03604a54bf533d4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 30 Oct 2015 16:04:50 -0700 Subject: [PATCH 17/23] wip on fixing GroupCoordinatorResponseTest --- .../kafka/coordinator/GroupCoordinator.scala | 24 +- .../coordinator/GroupMetadataManager.scala | 219 ++++++++++-------- .../GroupCoordinatorResponseTest.scala | 89 +++++-- 3 files changed, 203 insertions(+), 129 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index b111c93c7b41b..a6393a0956c36 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -273,10 +273,7 @@ class GroupCoordinator(val brokerId: Int, group.transitionTo(Stable) // persist the group metadata and upon finish propagate the assignment - val responseCode = groupManager.storeGroup(group, groupAssignment) - - // send the assignment back to members - propagateAssignment(group, groupAssignment, responseCode) + groupManager.storeGroup(group, groupAssignment) } case Stable => @@ -436,18 +433,6 @@ class GroupCoordinator(val brokerId: Int, errorCode=errorCode) } - def propagateAssignment(group: GroupMetadata, - assignment: Map[String, Array[Byte]], - errorCode: Short) { - for (member <- group.allMembers) { - member.assignment = assignment.getOrElse(member.memberId, Array.empty[Byte]) - if (member.awaitingSyncCallback != null) { - member.awaitingSyncCallback(member.assignment, errorCode) - member.awaitingSyncCallback = null - } - } - } - /** * Complete existing DelayedHeartbeats for the given member and schedule the next one */ @@ -500,12 +485,7 @@ class GroupCoordinator(val brokerId: Int, private def prepareRebalance(group: GroupMetadata) { // if any members are awaiting sync, cancel their request and have them rejoin if (group.is(AwaitingSync)) { - for (member <- group.allMembers) { - if (member.awaitingSyncCallback != null) { - member.awaitingSyncCallback(Array.empty, Errors.REBALANCE_IN_PROGRESS.code) - member.awaitingSyncCallback = null - } - } + groupManager.propagateAssignment(group, Errors.REBALANCE_IN_PROGRESS.code) } group.allMembers.foreach(_.assignment = null) diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index e5ca8fc5ed76a..10f317353d319 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -60,6 +60,9 @@ class GroupMetadataManager(val config: OffsetConfig, /* partitions of consumer groups that are being loaded */ private val loadingPartitions: mutable.Set[Int] = mutable.Set() + /* partitions of consumer groups that are assigned, using the same loading partition lock */ + private val ownedPartitions: mutable.Set[Int] = mutable.Set() + /* lock for expiring stale offsets */ private val offsetExpireLock = new ReentrantReadWriteLock() @@ -93,14 +96,14 @@ class GroupMetadataManager(val config: OffsetConfig, def partitionFor(group: String): Int = Utils.abs(group.hashCode) % groupMetadataTopicPartitionCount - def partitionLeaderIsLocal(partition: Int): Boolean = getHighWatermark(partition) != -1L + def partitionLeaderIsLocal(partition: Int): Boolean = loadingPartitions synchronized ownedPartitions.contains(partition) def partitionLoadingInProgress(partition: Int): Boolean = loadingPartitions synchronized loadingPartitions.contains(partition) /** * Get the group associated with the given groupId, or null if not found */ - def getGroup(groupId: String) = { + def getGroup(groupId: String): GroupMetadata = { groupsCache.get(groupId) } @@ -112,7 +115,7 @@ class GroupMetadataManager(val config: OffsetConfig, } private def addGroup(groupId: String, group: GroupMetadata): GroupMetadata = { - groupsCache.put(groupId, group) + groupsCache.putIfNotExists(groupId, group) group } @@ -127,7 +130,7 @@ class GroupMetadataManager(val config: OffsetConfig, } def storeGroup(group: GroupMetadata, - groupAssignment: Map[String, Array[Byte]]): Short = { + groupAssignment: Map[String, Array[Byte]]) { // construct the message to append val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), @@ -139,9 +142,6 @@ class GroupMetadataManager(val config: OffsetConfig, val groupMetadataMessageSet = Map(groupMetadataPartition -> new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message)) - val responseCode = Errors.NONE.code - val responseLatch = new CountDownLatch(1) - // set the callback function to insert offsets into cache after log append completed def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { // the append response should only contain the topics partition @@ -153,25 +153,39 @@ class GroupMetadataManager(val config: OffsetConfig, // the offset and metadata to cache if the append status has no error val status = responseStatus(groupMetadataPartition) + var responseCode = Errors.NONE.code if (status.error != ErrorMapping.NoError) { debug("Metadata from group %s with generation %d failed when appending to log due to %s" .format(group.groupId, group.generationId, ErrorMapping.exceptionNameFor(status.error))) // transform the log append error code to the corresponding the commit status error code - if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) - ErrorMapping.ConsumerCoordinatorNotAvailableCode - else if (status.error == ErrorMapping.NotLeaderForPartitionCode) - ErrorMapping.NotCoordinatorForConsumerCode - else if (status.error == ErrorMapping.MessageSizeTooLargeCode + responseCode = if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) { + Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code + } else if (status.error == ErrorMapping.NotLeaderForPartitionCode) { + Errors.NOT_COORDINATOR_FOR_GROUP.code + } else if (status.error == ErrorMapping.MessageSizeTooLargeCode || status.error == ErrorMapping.MessageSetSizeTooLargeCode - || status.error == ErrorMapping.InvalidFetchSizeCode) - Errors.INVALID_COMMIT_OFFSET_SIZE.code - else + || status.error == ErrorMapping.InvalidFetchSizeCode) { + + error("Appending metadata message for group %s generation %d failed due to %s, returning UNKNOWN error code to the client" + .format(group.groupId, group.generationId, ErrorMapping.exceptionNameFor(status.error))) + + Errors.UNKNOWN.code + } else { + + error("Appending metadata message for group %s generation %d failed due to unexpected error: %s" + .format(group.groupId, group.generationId, status.error)) + status.error + } + } + + for (member <- group.allMembers) { + member.assignment = groupAssignment.getOrElse(member.memberId, Array.empty[Byte]) } - // notify the storing thread to return - responseLatch.countDown() + // propagate the assignments + propagateAssignment(group, Errors.NONE.code) } // call replica manager to append the offset messages @@ -181,10 +195,17 @@ class GroupMetadataManager(val config: OffsetConfig, true, // allow appending to internal offset topic groupMetadataMessageSet, putCacheCallback) + } - responseLatch.await() - - responseCode + def propagateAssignment(group: GroupMetadata, + errorCode: Short) { + val hasError = errorCode == Errors.NONE.code + for (member <- group.allMembers) { + if (member.awaitingSyncCallback != null) { + member.awaitingSyncCallback(if (hasError) Array.empty else member.assignment, errorCode) + member.awaitingSyncCallback = null + } + } } /** @@ -313,6 +334,8 @@ class GroupMetadataManager(val config: OffsetConfig, val topicPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition) loadingPartitions synchronized { + ownedPartitions.add(offsetsPartition) + if (loadingPartitions.contains(offsetsPartition)) { info("Offset load from %s already in progress.".format(topicPartition)) } else { @@ -413,6 +436,8 @@ class GroupMetadataManager(val config: OffsetConfig, if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." .format(numRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) + + loadingPartitions synchronized ownedPartitions.remove(offsetsPartition) } /** @@ -532,18 +557,15 @@ class GroupMetadataManager(val config: OffsetConfig, object GroupMetadataManager { - private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) - - private val CURRENT_OFFSET_SCHEMA_VERSION = 1.toShort + private val CURRENT_OFFSET_KEY_SCHEMA_VERSION = 1.toShort + private val CURRENT_GROUP_KEY_SCHEMA_VERSION = 2.toShort - private val CURRENT_GROUP_SCHEMA_VERSION = 2.toShort - - private val OFFSET_COMMIT_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING), + private val OFFSET_COMMIT_KEY_SCHEMA = new Schema(new Field("group", STRING), new Field("topic", STRING), new Field("partition", INT32)) - private val OFFSET_KEY_GROUP_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("group") - private val OFFSET_KEY_TOPIC_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("topic") - private val OFFSET_KEY_PARTITION_FIELD = OFFSET_COMMIT_KEY_SCHEMA_V0.get("partition") + private val OFFSET_KEY_GROUP_FIELD = OFFSET_COMMIT_KEY_SCHEMA.get("group") + private val OFFSET_KEY_TOPIC_FIELD = OFFSET_COMMIT_KEY_SCHEMA.get("topic") + private val OFFSET_KEY_PARTITION_FIELD = OFFSET_COMMIT_KEY_SCHEMA.get("partition") private val OFFSET_COMMIT_VALUE_SCHEMA_V0 = new Schema(new Field("offset", INT64), new Field("metadata", STRING, "Associated metadata.", ""), @@ -562,8 +584,8 @@ object GroupMetadataManager { private val OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") private val OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") - private val GROUP_METADATA_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING)) - private val GROUP_KEY_GROUP_FIELD = GROUP_METADATA_KEY_SCHEMA_V0.get("group") + private val GROUP_METADATA_KEY_SCHEMA = new Schema(new Field("group", STRING)) + private val GROUP_KEY_GROUP_FIELD = GROUP_METADATA_KEY_SCHEMA.get("group") private val MEMBER_METADATA_V0 = new Schema(new Field("member_id", STRING), new Field("session_timeout", INT32), @@ -586,16 +608,44 @@ object GroupMetadataManager { private val GROUP_METADATA_LEADER_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("leader") private val GROUP_METADATA_MEMBERS_V0 = GROUP_METADATA_VALUE_SCHEMA_V0.get("members") - // map of versions to schemas - private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), - 1 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V1), - 2 -> KeyAndValueSchemas(GROUP_METADATA_KEY_SCHEMA_V0, GROUP_METADATA_VALUE_SCHEMA_V0)) + // map of versions to key schemas as data types + private val MESSAGE_TYPE_SCHEMAS = Map(0 -> OFFSET_COMMIT_KEY_SCHEMA, + 1 -> OFFSET_COMMIT_KEY_SCHEMA, + 2 -> GROUP_METADATA_KEY_SCHEMA) + + // map of version of offset value schemas + private val OFFSET_VALUE_SCHEMAS = Map(0 -> OFFSET_COMMIT_VALUE_SCHEMA_V0, + 1 -> OFFSET_COMMIT_VALUE_SCHEMA_V1) + private val CURRENT_OFFSET_VALUE_SCHEMA_VERSION = 1.toShort + + // map of version of group metadata value schemas + private val GROUP_VALUE_SCHEMAS = Map(0 -> GROUP_METADATA_VALUE_SCHEMA_V0) + private val CURRENT_GROUP_VALUE_SCHEMA_VERSION = 0.toShort - private val CURRENT_OFFSET_SCHEMA = schemaFor(CURRENT_OFFSET_SCHEMA_VERSION) - private val CURRENT_GROUP_SCHEMA = schemaFor(CURRENT_GROUP_SCHEMA_VERSION) + private val CURRENT_OFFSET_KEY_SCHEMA = schemaForKey(CURRENT_OFFSET_KEY_SCHEMA_VERSION) + private val CURRENT_GROUP_KEY_SCHEMA = schemaForKey(CURRENT_GROUP_KEY_SCHEMA_VERSION) + + private val CURRENT_OFFSET_VALUE_SCHEMA = schemaForOffset(CURRENT_OFFSET_VALUE_SCHEMA_VERSION) + private val CURRENT_GROUP_VALUE_SCHEMA = schemaForGroup(CURRENT_GROUP_VALUE_SCHEMA_VERSION) + + private def schemaForKey(version: Int) = { + val schemaOpt = MESSAGE_TYPE_SCHEMAS.get(version) + schemaOpt match { + case Some(schema) => schema + case _ => throw new KafkaException("Unknown offset schema version " + version) + } + } - private def schemaFor(version: Int) = { - val schemaOpt = OFFSET_SCHEMAS.get(version) + private def schemaForOffset(version: Int) = { + val schemaOpt = OFFSET_VALUE_SCHEMAS.get(version) + schemaOpt match { + case Some(schema) => schema + case _ => throw new KafkaException("Unknown offset schema version " + version) + } + } + + private def schemaForGroup(version: Int) = { + val schemaOpt = GROUP_VALUE_SCHEMAS.get(version) schemaOpt match { case Some(schema) => schema case _ => throw new KafkaException("Unknown offset schema version " + version) @@ -608,13 +658,13 @@ object GroupMetadataManager { * @return key for offset commit message */ private def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { - val key = new Struct(CURRENT_OFFSET_SCHEMA.keySchema) + val key = new Struct(CURRENT_OFFSET_KEY_SCHEMA) key.set(OFFSET_KEY_GROUP_FIELD, group) key.set(OFFSET_KEY_TOPIC_FIELD, topic) key.set(OFFSET_KEY_PARTITION_FIELD, partition) val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf) - byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + byteBuffer.putShort(CURRENT_OFFSET_KEY_SCHEMA_VERSION) key.writeTo(byteBuffer) byteBuffer.array() } @@ -625,11 +675,11 @@ object GroupMetadataManager { * @return key bytes for group metadata message */ private def groupMetadataKey(group: String): Array[Byte] = { - val key = new Struct(CURRENT_GROUP_SCHEMA.keySchema) + val key = new Struct(CURRENT_GROUP_KEY_SCHEMA) key.set(GROUP_KEY_GROUP_FIELD, group) val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf) - byteBuffer.putShort(CURRENT_GROUP_SCHEMA_VERSION) + byteBuffer.putShort(CURRENT_GROUP_KEY_SCHEMA_VERSION) key.writeTo(byteBuffer) byteBuffer.array() } @@ -642,13 +692,13 @@ object GroupMetadataManager { */ private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { // generate commit value with schema version 1 - val value = new Struct(CURRENT_OFFSET_SCHEMA.valueSchema) + val value = new Struct(CURRENT_OFFSET_VALUE_SCHEMA) value.set(OFFSET_VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) value.set(OFFSET_VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) value.set(OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1, offsetAndMetadata.commitTimestamp) value.set(OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.expireTimestamp) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) - byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + byteBuffer.putShort(CURRENT_OFFSET_KEY_SCHEMA_VERSION) value.writeTo(byteBuffer) byteBuffer.array() } @@ -662,7 +712,7 @@ object GroupMetadataManager { */ private def groupMetadataValue(groupMetadata: GroupMetadata, assignment: Map[String, Array[Byte]]): Array[Byte] = { // generate commit value with schema version 1 - val value = new Struct(CURRENT_GROUP_SCHEMA.valueSchema) + val value = new Struct(CURRENT_GROUP_VALUE_SCHEMA) value.set(GROUP_METADATA_PROTOCOL_TYPE_V0, groupMetadata.protocolType) value.set(GROUP_METADATA_GENERATION_V0, groupMetadata.generationId) value.set(GROUP_METADATA_PROTOCOL_V0, groupMetadata.protocol) @@ -674,10 +724,8 @@ object GroupMetadataManager { memberStruct.set(MEMBER_METADATA_MEMBER_ID_V0, memberMetadata.memberId) memberStruct.set(MEMBER_METADATA_SESSION_TIMEOUT_V0, memberMetadata.sessionTimeoutMs) - val selectedProtocol = memberMetadata.supportedProtocols.filter(_._1.equals(groupMetadata.protocol)) - assert(selectedProtocol.length == 1) - - memberStruct.set(MEMBER_METADATA_SUBSCRIPTION_V0, ByteBuffer.wrap(selectedProtocol.head._2)) + val metadata = memberMetadata.metadata(groupMetadata.protocol) + memberStruct.set(MEMBER_METADATA_SUBSCRIPTION_V0, ByteBuffer.wrap(metadata)) val memberAssignment = assignment(memberMetadata.memberId) assert(memberAssignment != null) @@ -690,7 +738,7 @@ object GroupMetadataManager { value.set(GROUP_METADATA_MEMBERS_V0, memberArray.toArray) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) - byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) + byteBuffer.putShort(CURRENT_OFFSET_KEY_SCHEMA_VERSION) value.writeTo(byteBuffer) byteBuffer.array() } @@ -703,10 +751,10 @@ object GroupMetadataManager { */ private def readMessageKey(buffer: ByteBuffer): BaseKey = { val version = buffer.getShort - val keySchema = schemaFor(version).keySchema + val keySchema = schemaForKey(version) val key = keySchema.read(buffer).asInstanceOf[Struct] - if (version <= CURRENT_OFFSET_SCHEMA_VERSION) { + if (version <= CURRENT_OFFSET_KEY_SCHEMA_VERSION) { // version 0 and 1 refer to offset val group = key.get(OFFSET_KEY_GROUP_FIELD).asInstanceOf[String] val topic = key.get(OFFSET_KEY_TOPIC_FIELD).asInstanceOf[String] @@ -714,7 +762,7 @@ object GroupMetadataManager { OffsetKey(version, GroupTopicPartition(group, TopicAndPartition(topic, partition))) - } else if (version == CURRENT_GROUP_SCHEMA_VERSION) { + } else if (version == CURRENT_GROUP_KEY_SCHEMA_VERSION) { // version 2 refers to offset val group = key.get(GROUP_KEY_GROUP_FIELD).asInstanceOf[String] @@ -731,22 +779,24 @@ object GroupMetadataManager { * @return an offset-metadata object from the message */ private def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { - val structAndVersion = readMessageValueStruct(buffer) - - if (structAndVersion.value == null) { // tombstone + if(buffer == null) { // tombstone null } else { - if (structAndVersion.version == 0) { - val offset = structAndVersion.value.get(OFFSET_VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] - val metadata = structAndVersion.value.get(OFFSET_VALUE_METADATA_FIELD_V0).asInstanceOf[String] - val timestamp = structAndVersion.value.get(OFFSET_VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] + val version = buffer.getShort + val valueSchema = schemaForOffset(version) + val value = valueSchema.read(buffer).asInstanceOf[Struct] + + if (version == 0) { + val offset = value.get(OFFSET_VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] + val metadata = value.get(OFFSET_VALUE_METADATA_FIELD_V0).asInstanceOf[String] + val timestamp = value.get(OFFSET_VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] OffsetAndMetadata(offset, metadata, timestamp) - } else if (structAndVersion.version == 1) { - val offset = structAndVersion.value.get(OFFSET_VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] - val metadata = structAndVersion.value.get(OFFSET_VALUE_METADATA_FIELD_V1).asInstanceOf[String] - val commitTimestamp = structAndVersion.value.get(OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] - val expireTimestamp = structAndVersion.value.get(OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + } else if (version == 1) { + val offset = value.get(OFFSET_VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] + val metadata = value.get(OFFSET_VALUE_METADATA_FIELD_V1).asInstanceOf[String] + val commitTimestamp = value.get(OFFSET_VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + val expireTimestamp = value.get(OFFSET_VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] OffsetAndMetadata(offset, metadata, commitTimestamp, expireTimestamp) } else { @@ -762,21 +812,23 @@ object GroupMetadataManager { * @return a group metadata object from the message */ private def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = { - val structAndVersion = readMessageValueStruct(buffer) - - if (structAndVersion.value == null) { // tombstone + if(buffer == null) { // tombstone null } else { - if (structAndVersion.version == CURRENT_GROUP_SCHEMA_VERSION) { - val protocolType = structAndVersion.value.get(GROUP_METADATA_PROTOCOL_TYPE_V0).asInstanceOf[String] + val version = buffer.getShort + val valueSchema = schemaForGroup(version) + val value = valueSchema.read(buffer).asInstanceOf[Struct] + + if (version == 0) { + val protocolType = value.get(GROUP_METADATA_PROTOCOL_TYPE_V0).asInstanceOf[String] val group = new GroupMetadata(groupId, protocolType) - group.generationId = structAndVersion.value.get(GROUP_METADATA_GENERATION_V0).asInstanceOf[Int] - group.leaderId = structAndVersion.value.get(GROUP_METADATA_LEADER_V0).asInstanceOf[String] - group.protocol = structAndVersion.value.get(GROUP_METADATA_PROTOCOL_V0).asInstanceOf[String] + group.generationId = value.get(GROUP_METADATA_GENERATION_V0).asInstanceOf[Int] + group.leaderId = value.get(GROUP_METADATA_LEADER_V0).asInstanceOf[String] + group.protocol = value.get(GROUP_METADATA_PROTOCOL_V0).asInstanceOf[String] - structAndVersion.value.getArray(GROUP_METADATA_MEMBERS_V0).foreach { + value.getArray(GROUP_METADATA_MEMBERS_V0).foreach { case memberMetadataObj => val memberMetadata = memberMetadataObj.asInstanceOf[Struct] val memberId = memberMetadata.get(MEMBER_METADATA_MEMBER_ID_V0).asInstanceOf[String] @@ -797,35 +849,20 @@ object GroupMetadataManager { } } - private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { - if(buffer == null) { // tombstone - MessageValueStructAndVersion(null, -1) - } else { - val version = buffer.getShort - val valueSchema = schemaFor(version).valueSchema - val value = valueSchema.read(buffer).asInstanceOf[Struct] - - MessageValueStructAndVersion(value, version) - } - } - // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. // (specify --formatter "kafka.coordinator.GroupMetadataManager\$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 GroupMetadataManager.readMessageKey(ByteBuffer.wrap(key)).toString - val formattedValue = if (value == null) "NULL" else GroupMetadataManager.readMessageValueStruct(ByteBuffer.wrap(value)).value.toString + val formattedValue = if (value == null) "NULL" else GroupMetadataManager.readOffsetMessageValue(ByteBuffer.wrap(value)).toString output.write(formattedKey.getBytes) output.write("::".getBytes) output.write(formattedValue.getBytes) output.write("\n".getBytes) } } - } -case class MessageValueStructAndVersion(value: Struct, version: Short) - case class GroupTopicPartition(group: String, topicPartition: TopicAndPartition) { def this(group: String, topic: String, partition: Int) = diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 4fe0b47d56c96..990fc10630fa1 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -53,14 +53,27 @@ class GroupCoordinatorResponseTest extends JUnitSuite { var consumerCoordinator: GroupCoordinator = null var groupManager : GroupMetadataManager = null - final val groupId = "groupId" - final val protocolType = "consumer" - final val memberId = "memberId" - final val metadata = Array[Byte]() - final val protocols = List(("range", metadata)) + private val groupId = "groupId" + private val protocolType = "consumer" + private val memberId = "memberId" + private val metadata = Array[Byte]() + private val protocols = List(("range", metadata)) private val groupsCache = new Pool[String, GroupMetadata] + private val propagateRebalanceAnswer = new IAnswer[Unit] { + override def answer = { + for (member <- groupsCache.get(groupId).allMembers) { + if (member.awaitingSyncCallback != null) { + member.awaitingSyncCallback.apply(Array.empty[Byte], Errors.REBALANCE_IN_PROGRESS.code) + member.awaitingSyncCallback = null + } + } + } + } + + private var memberIndex : Int = 1 + @Before def setUp() { val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") @@ -205,11 +218,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, isCoordinatorForGroup = true) + val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.memberId, Map.empty, true) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()), true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) @@ -244,7 +258,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, joinGroupErrorCode) EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map.empty, true) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()), true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) @@ -265,7 +279,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, joinGroupErrorCode) EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) + val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()), true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) @@ -301,7 +315,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, joinGroupResult.errorCode) EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true) + val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()), true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) @@ -323,7 +337,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.reset(groupManager) // send the sync group with an invalid generation - val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map.empty, true) + val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()), true) assertEquals(Errors.ILLEGAL_GENERATION.code, syncGroupResult._2) } @@ -341,7 +355,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, firstJoinResult.errorCode) EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) assertEquals(Errors.NONE.code, firstSyncResult._2) EasyMock.reset(groupManager) @@ -382,7 +396,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, firstJoinResult.errorCode) EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) assertEquals(Errors.NONE.code, firstSyncResult._2) // join groups from the leader should force the group to rebalance, which allows the @@ -410,7 +424,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, firstJoinResult.errorCode) EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) assertEquals(Errors.NONE.code, firstSyncResult._2) EasyMock.reset(groupManager) @@ -455,7 +469,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, firstJoinResult.errorCode) EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) assertEquals(Errors.NONE.code, firstSyncResult._2) EasyMock.reset(groupManager) @@ -508,7 +522,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, joinGroupResult.errorCode) EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true) + val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) @@ -718,11 +732,16 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) if (!groupsCache.contains(groupId)) { val groupMetadata = new GroupMetadata(groupId, protocolType) + + + groupMetadata.add(memberId, new MemberMetadata(memberId, groupId, sessionTimeout, protocols)) groupsCache.putIfNotExists(groupId, groupMetadata) EasyMock.expect(groupManager.addGroup(groupId, protocolType)).andReturn(groupMetadata) } + EasyMock.expect(groupManager.propagateAssignment(groupsCache.get(groupId), Errors.REBALANCE_IN_PROGRESS.code)) + .andAnswer(propagateRebalanceAnswer) EasyMock.replay(groupManager) @@ -738,12 +757,31 @@ class GroupCoordinatorResponseTest extends JUnitSuite { isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback + val storeGroupAnswer = new IAnswer[Unit] { + override def answer = { + for (member <- groupsCache.get(groupId).allMembers) { + member.assignment = assignment.getOrElse(member.memberId, Array.empty[Byte]) + if (member.awaitingSyncCallback != null) { + member.awaitingSyncCallback.apply(member.assignment, Errors.NONE.code) + member.awaitingSyncCallback = null + } + } + } + } + EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) if (groupsCache.contains(groupId)) { - EasyMock.expect(groupManager.storeGroup(groupsCache.get(groupId), assignment)).andReturn(Errors.NONE.code) + val members = groupsCache.get(groupId).allMembers.filter(_.memberId.equals(leaderId)) + if (members.length != 1) { + responseCallback.apply(null, Errors.UNKNOWN_MEMBER_ID.code) + } else { + members.head.awaitingSyncCallback = responseCallback + } } + + EasyMock.expect(groupManager.storeGroup(groupsCache.get(groupId), assignment)).andAnswer(storeGroupAnswer) EasyMock.replay(groupManager) consumerCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) @@ -759,6 +797,23 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + if (groupsCache.contains(groupId)) { + val members = groupsCache.get(groupId).allMembers.filter(_.memberId.equals(memberId)) + if (members.length != 1) { + responseCallback.apply(null, Errors.UNKNOWN_MEMBER_ID.code) + } else { + val member = members.head + if (member.assignment != null) { + responseCallback.apply(member.assignment, Errors.NONE.code) + } else { + member.awaitingSyncCallback = responseCallback + } + } + } else { + responseCallback.apply(null, Errors.UNKNOWN_MEMBER_ID.code) + } + EasyMock.expect(groupManager.propagateAssignment(groupsCache.get(groupId), Errors.REBALANCE_IN_PROGRESS.code)) + .andAnswer(propagateRebalanceAnswer) EasyMock.replay(groupManager) consumerCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback) @@ -847,6 +902,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) + EasyMock.expect(groupManager.propagateAssignment(groupsCache.get(groupId), Errors.REBALANCE_IN_PROGRESS.code)) + .andAnswer(propagateRebalanceAnswer) EasyMock.expect(groupManager.removeGroup(groupId)) EasyMock.replay(groupManager) From f308a033d2975450e06e876dcc92767ccb971ee4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 30 Oct 2015 16:15:31 -0700 Subject: [PATCH 18/23] address comments in github --- .../scala/kafka/coordinator/GroupCoordinator.scala | 12 ++++++------ .../kafka/coordinator/GroupMetadataManager.scala | 5 +++-- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index a6393a0956c36..f4e80b08bfec6 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.requests.JoinGroupRequest import scala.collection.{Map, Seq, immutable} -case class GroupManagerConfig(groupMinSessionTimeoutMs: Int, +case class GroupConfig(groupMinSessionTimeoutMs: Int, groupMaxSessionTimeoutMs: Int) case class JoinGroupResult(members: Map[String, Array[Byte]], @@ -46,7 +46,7 @@ case class JoinGroupResult(members: Map[String, Array[Byte]], * groups. Groups are assigned to coordinators based on their group names. */ class GroupCoordinator(val brokerId: Int, - val groupConfig: GroupManagerConfig, + val groupConfig: GroupConfig, val offsetConfig: OffsetConfig, private val groupManager: GroupMetadataManager) extends Logging { type JoinCallback = JoinGroupResult => Unit @@ -60,12 +60,12 @@ class GroupCoordinator(val brokerId: Int, private var joinPurgatory: DelayedOperationPurgatory[DelayedJoin] = null def this(brokerId: Int, - groupConfig: GroupManagerConfig, + groupConfig: GroupConfig, offsetConfig: OffsetConfig, replicaManager: ReplicaManager, zkUtils: ZkUtils, scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig, - new GroupMetadataManager(offsetConfig, replicaManager, zkUtils, scheduler)) + new GroupMetadataManager(brokerId, offsetConfig, replicaManager, zkUtils, scheduler)) def offsetsTopicConfigs: Properties = { val props = new Properties @@ -609,7 +609,7 @@ object GroupCoordinator { offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) - val groupConfig = GroupManagerConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, + val groupConfig = GroupConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs) new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkUtils, kafkaScheduler) @@ -625,7 +625,7 @@ object GroupCoordinator { offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) - val groupConfig = GroupManagerConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, + val groupConfig = GroupConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs) new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager) diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 10f317353d319..aa5dde4a671ec 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -46,7 +46,8 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} import com.yammer.metrics.core.Gauge -class GroupMetadataManager(val config: OffsetConfig, +class GroupMetadataManager(val brokerId: Int, + val config: OffsetConfig, replicaManager: ReplicaManager, zkUtils: ZkUtils, scheduler: Scheduler) extends Logging with KafkaMetricsGroup { @@ -75,7 +76,7 @@ class GroupMetadataManager(val config: OffsetConfig, /* number of partitions for the consumer metadata topic */ private val groupMetadataTopicPartitionCount = getOffsetsTopicPartitionCount - this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " + this.logIdent = "[Group Metadata Manager on Broker " + brokerId + "]: " scheduler.schedule(name = "delete-expired-consumer-offsets", fun = deleteExpiredOffsets, diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index c6ea84ec36197..f79cc1d9bbdab 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -50,7 +50,7 @@ import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBroker import kafka.network.{BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge -import kafka.coordinator.{GroupManagerConfig, GroupCoordinator} +import kafka.coordinator.{GroupConfig, GroupCoordinator} object KafkaServer { // Copy the subset of properties that are relevant to Logs From ca49e790a4fb0d522c60e52f1e84b6e05bed95e3 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sun, 1 Nov 2015 01:47:24 -0700 Subject: [PATCH 19/23] wip to use easyMock.capture --- .../kafka/coordinator/GroupCoordinator.scala | 8 +- .../coordinator/GroupMetadataManager.scala | 41 +- .../GroupCoordinatorResponseTest.scala | 462 +++++++----------- 3 files changed, 228 insertions(+), 283 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index f4e80b08bfec6..7144a0833011c 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -48,7 +48,7 @@ case class JoinGroupResult(members: Map[String, Array[Byte]], class GroupCoordinator(val brokerId: Int, val groupConfig: GroupConfig, val offsetConfig: OffsetConfig, - private val groupManager: GroupMetadataManager) extends Logging { + val groupManager: GroupMetadataManager) extends Logging { type JoinCallback = JoinGroupResult => Unit type SyncCallback = (Array[Byte], Short) => Unit @@ -64,7 +64,7 @@ class GroupCoordinator(val brokerId: Int, offsetConfig: OffsetConfig, replicaManager: ReplicaManager, zkUtils: ZkUtils, - scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig, + scheduler: Scheduler) = this(brokerId, groupConfig, offsetConfig, new GroupMetadataManager(brokerId, offsetConfig, replicaManager, zkUtils, scheduler)) def offsetsTopicConfigs: Properties = { @@ -600,7 +600,7 @@ object GroupCoordinator { def create(config: KafkaConfig, zkUtils: ZkUtils, replicaManager: ReplicaManager, - kafkaScheduler: KafkaScheduler): GroupCoordinator = { + scheduler: Scheduler): GroupCoordinator = { val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize, loadBufferSize = config.offsetsLoadBufferSize, offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, @@ -612,7 +612,7 @@ object GroupCoordinator { val groupConfig = GroupConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs, groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs) - new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkUtils, kafkaScheduler) + new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkUtils, scheduler) } def create(config: KafkaConfig, diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index aa5dde4a671ec..2b0b3accf92b6 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -121,13 +121,37 @@ class GroupMetadataManager(val brokerId: Int, } /** - * Remove all metadata associated with the group, including its topics + * Remove all metadata associated with the group, note this function needs to be + * called inside the group lock * @param groupId the groupId of the group we are removing */ def removeGroup(groupId: String) { - if (!groupsCache.contains(groupId)) + if (groupsCache.remove(groupId) == null) throw new IllegalArgumentException("Cannot remove non-existing group") - groupsCache.remove(groupId) + + // Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say, + // if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and + // retry removing this group. + val groupPartition = partitionFor(groupId) + val tomstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(groupId)) + + val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) + partitionOpt.foreach { partition => + val appendPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) + + trace("Marking group %s as deleted.".format(groupId)) + + try { + // do not need to require acks since even if the tombsone is lost, + // it will be appended again by the new leader + // TODO: have periodic purging instead of immediate removal of groups + partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, tomstone)) + } catch { + case t: Throwable => + error("Failed to mark group %s as deleted in %s.".format(groupId, appendPartition), t) + // ignore and continue + } + } } def storeGroup(group: GroupMetadata, @@ -553,6 +577,17 @@ class GroupMetadataManager(val brokerId: Int, else config.offsetsTopicNumPartitions } + + /** + * Add the partition into the owned list + * + * NOTE: this is for test only + */ + def addPartitionOwnership(partition: Int) { + loadingPartitions synchronized { + ownedPartitions.add(partition) + } + } } diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 990fc10630fa1..2acb871355604 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -18,18 +18,22 @@ package kafka.coordinator -import java.util.concurrent.TimeUnit - import org.junit.Assert._ + +import kafka.api.ProducerResponseStatus import kafka.common.{OffsetAndMetadata, TopicAndPartition} -import kafka.server.KafkaConfig -import kafka.utils.{Pool, TestUtils} +import kafka.message.MessageSet +import kafka.server.{ReplicaManager, KafkaConfig} +import kafka.utils._ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest} -import org.easymock.{IAnswer, EasyMock} +import org.easymock.{Capture, IAnswer, EasyMock} import org.junit.{After, Before, Test} import org.scalatest.junit.JUnitSuite +import java.util.concurrent.TimeUnit + +import scala.collection._ import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future, Promise} @@ -51,45 +55,47 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val ConsumerMaxSessionTimeout = 1000 val DefaultSessionTimeout = 500 var consumerCoordinator: GroupCoordinator = null - var groupManager : GroupMetadataManager = null + var replicaManager: ReplicaManager = null + var scheduler: KafkaScheduler = null + var zkUtils: ZkUtils = null private val groupId = "groupId" private val protocolType = "consumer" private val memberId = "memberId" private val metadata = Array[Byte]() private val protocols = List(("range", metadata)) + private var groupPartitionId: Int = -1 - private val groupsCache = new Pool[String, GroupMetadata] - - private val propagateRebalanceAnswer = new IAnswer[Unit] { - override def answer = { - for (member <- groupsCache.get(groupId).allMembers) { - if (member.awaitingSyncCallback != null) { - member.awaitingSyncCallback.apply(Array.empty[Byte], Errors.REBALANCE_IN_PROGRESS.code) - member.awaitingSyncCallback = null - } - } - } - } - - private var memberIndex : Int = 1 + // we use this string value since its hashcode % #.partitions is different + private val otherGroupId = "otherGroup" @Before def setUp() { val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) - - groupManager = EasyMock.createStrictMock(classOf[GroupMetadataManager]) - - consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), groupManager) + + // make two partitions of the group topic to make sure some partitions are not owned by the coordinator + val ret = mutable.Map[String, Map[Int, Seq[Int]]]() + ret += (GroupCoordinator.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1))) + + replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) + + zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) + EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(GroupCoordinator.GroupMetadataTopicName))).andReturn(ret) + EasyMock.replay(zkUtils) + + consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), zkUtils, replicaManager, new MockScheduler(new MockTime())) consumerCoordinator.startup() + + // add the partition into the owned partition list + groupPartitionId = consumerCoordinator.partitionFor(groupId) + consumerCoordinator.groupManager.addPartitionOwnership(groupPartitionId) } @After def tearDown() { - groupsCache.clear() - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) consumerCoordinator.shutdown() } @@ -97,8 +103,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testJoinGroupWrongCoordinator() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, - protocols, isCoordinatorForGroup = false) + val joinGroupResult = joinGroup(otherGroupId, memberId, DefaultSessionTimeout, protocolType, + protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, joinGroupErrorCode) } @@ -107,8 +113,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testJoinGroupSessionTimeoutTooSmall() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, ConsumerMinSessionTimeout - 1, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, ConsumerMinSessionTimeout - 1, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) } @@ -117,16 +122,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testJoinGroupSessionTimeoutTooLarge() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, ConsumerMaxSessionTimeout + 1, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, ConsumerMaxSessionTimeout + 1, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode) } @Test def testJoinGroupUnknownConsumerNewGroup() { - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.UNKNOWN_MEMBER_ID.code, joinGroupErrorCode) } @@ -137,7 +140,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, - protocols, isCoordinatorForGroup = true) + protocols) assertEquals(Errors.INVALID_GROUP_ID.code, joinGroupResult.errorCode) } @@ -146,7 +149,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, - protocols, isCoordinatorForGroup = true) + protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) } @@ -157,12 +160,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, - protocols, isCoordinatorForGroup = true) + protocols) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, "copycat", - protocols, isCoordinatorForGroup = true) + protocols) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) } @@ -172,13 +175,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, List(("range", metadata)), - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, List(("range", metadata))) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, - List(("roundrobin", metadata)), isCoordinatorForGroup = true) + List(("roundrobin", metadata))) assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode) } @@ -187,27 +189,25 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "memberId" - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) - val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, otherJoinGroupResult.errorCode) } @Test def testHeartbeatWrongCoordinator() { - val heartbeatResult = heartbeat(groupId, memberId, -1, isCoordinatorForGroup = false) + val heartbeatResult = heartbeat(otherGroupId, memberId, -1) assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, heartbeatResult) } @Test def testHeartbeatUnknownGroup() { - val heartbeatResult = heartbeat(groupId, memberId, -1, isCoordinatorForGroup = true) + val heartbeatResult = heartbeat(groupId, memberId, -1) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult) } @@ -216,19 +216,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "memberId" - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]())) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) - val heartbeatResult = heartbeat(groupId, otherMemberId, 1, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val heartbeatResult = heartbeat(groupId, otherMemberId, 1) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult) } @@ -236,14 +235,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testHeartbeatRebalanceInProgress() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val heartbeatResult = heartbeat(groupId, assignedMemberId, 2) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @@ -251,19 +249,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testHeartbeatIllegalGeneration() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]())) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) - val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val heartbeatResult = heartbeat(groupId, assignedMemberId, 2) assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult) } @@ -271,20 +268,19 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testValidHeartbeat() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]())) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) - val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1) assertEquals(Errors.NONE.code, heartbeatResult) } @@ -292,7 +288,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testSyncGroupNotCoordinator() { val generation = 1 - val syncGroupResult = syncGroupFollower(groupId, generation, memberId, false) + val syncGroupResult = syncGroupFollower(otherGroupId, generation, memberId) assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, syncGroupResult._2) } @@ -300,7 +296,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testSyncGroupFromUnknownGroup() { val generation = 1 - val syncGroupResult = syncGroupFollower(groupId, generation, memberId, true) + val syncGroupResult = syncGroupFollower(groupId, generation, memberId) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, syncGroupResult._2) } @@ -308,20 +304,19 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testSyncGroupFromUnknownMember() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]())) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val unknownMemberId = "blah" - val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId, true) + val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, unknownMemberSyncResult._2) } @@ -329,15 +324,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testSyncGroupFromIllegalGeneration() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val generationId = joinGroupResult.generationId assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) // send the sync group with an invalid generation - val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()), true) + val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]())) assertEquals(Errors.ILLEGAL_GENERATION.code, syncGroupResult._2) } @@ -348,23 +342,22 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 2. join and sync with the first member and a new member val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]())) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) - EasyMock.reset(groupManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -378,9 +371,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val nextGenerationId = joinResult.generationId // this shouldn't cause a rebalance since protocol information hasn't changed - EasyMock.reset(groupManager) - val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, DefaultSessionTimeout, protocolType, protocols) assertEquals(Errors.NONE.code, followerJoinResult.errorCode) assertEquals(nextGenerationId, followerJoinResult.generationId) @@ -389,22 +381,21 @@ class GroupCoordinatorResponseTest extends JUnitSuite { @Test def testJoinGroupFromUnchangedLeaderShouldRebalance() { val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]())) assertEquals(Errors.NONE.code, firstSyncResult._2) // join groups from the leader should force the group to rebalance, which allows the // leader to push new assignments when local metadata changes - EasyMock.reset(groupManager) - val secondJoinResult = joinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val secondJoinResult = joinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) assertEquals(Errors.NONE.code, secondJoinResult.errorCode) assertNotEquals(firstGenerationId, secondJoinResult.generationId) @@ -417,23 +408,22 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 2. join and sync with the first member and a new member val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]())) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) - EasyMock.reset(groupManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -448,9 +438,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // with no leader SyncGroup, the follower's request should failure with an error indicating // that it should rejoin - EasyMock.reset(groupManager) - val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId) val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, followerSyncResult._2) } @@ -462,23 +451,22 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 2. join and sync with the first member and a new member val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) val firstMemberId = firstJoinResult.memberId val firstGenerationId = firstJoinResult.generationId assertEquals(firstMemberId, firstJoinResult.leaderId) assertEquals(Errors.NONE.code, firstJoinResult.errorCode) - EasyMock.reset(groupManager) - val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]())) assertEquals(Errors.NONE.code, firstSyncResult._2) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) - EasyMock.reset(groupManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -495,15 +483,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val followerId = otherJoinResult.memberId val followerAssignment = Array[Byte](1) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, - Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) + Map(leaderId -> leaderAssignment, followerId -> followerAssignment)) assertEquals(Errors.NONE.code, leaderSyncResult._2) assertEquals(leaderAssignment, leaderSyncResult._1) - EasyMock.reset(groupManager) - val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId) assertEquals(Errors.NONE.code, followerSyncResult._2) assertEquals(followerAssignment, followerSyncResult._1) } @@ -515,24 +502,23 @@ class GroupCoordinatorResponseTest extends JUnitSuite { // 2. join and sync with the first member and a new member val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) val firstMemberId = joinGroupResult.memberId val firstGenerationId = joinGroupResult.generationId assertEquals(firstMemberId, joinGroupResult.leaderId) assertEquals(Errors.NONE.code, joinGroupResult.errorCode) - EasyMock.reset(groupManager) - val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()), true) + EasyMock.reset(replicaManager) + val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]())) val syncGroupErrorCode = syncGroupResult._2 assertEquals(Errors.NONE.code, syncGroupErrorCode) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) - EasyMock.reset(groupManager) - val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols) val joinResult = await(joinFuture, DefaultSessionTimeout+100) val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100) @@ -549,12 +535,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(firstMemberId, joinResult.leaderId) assertEquals(firstMemberId, otherJoinResult.leaderId) - EasyMock.reset(groupManager) - val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId) - EasyMock.reset(groupManager) + EasyMock.reset(replicaManager) val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId, - Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true) + Map(leaderId -> leaderAssignment, followerId -> followerAssignment)) assertEquals(Errors.NONE.code, leaderSyncResult._2) assertEquals(leaderAssignment, leaderSyncResult._1) @@ -569,7 +555,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val tp = new TopicAndPartition("topic", 0) val offset = OffsetAndMetadata(0) - val commitOffsetResult = commitOffsets(groupId, memberId, generationId, Map(tp -> offset), true) + val commitOffsetResult = commitOffsets(groupId, memberId, generationId, immutable.Map(tp -> offset)) assertEquals(Errors.ILLEGAL_GENERATION.code, commitOffsetResult(tp)) } @@ -579,7 +565,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val offset = OffsetAndMetadata(0) val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, - OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset), true) + OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset)) assertEquals(Errors.NONE.code, commitOffsetResult(tp)) } @@ -589,15 +575,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val tp = new TopicAndPartition("topic", 0) val offset = OffsetAndMetadata(0) - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val generationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, Map(tp -> offset), true) + EasyMock.reset(replicaManager) + val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, immutable.Map(tp -> offset)) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, commitOffsetResult(tp)) } @@ -605,20 +590,19 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testHeartbeatDuringRebalanceCausesRebalanceInProgress() { // First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts) val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, - protocolType, protocols, isCoordinatorForGroup = true) + protocolType, protocols) val assignedConsumerId = joinGroupResult.memberId val initialGenerationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) // Then join with a new consumer to trigger a rebalance - EasyMock.reset(groupManager) - sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols) // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress - EasyMock.reset(groupManager) - val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId) assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult) } @@ -627,16 +611,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val initialGenerationId = joinGroupResult.generationId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(1, initialGenerationId) assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols) val nextGenerationId = otherJoinGroupResult.generationId val otherJoinGroupErrorCode = otherJoinGroupResult.errorCode assertEquals(2, nextGenerationId) @@ -647,14 +629,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testLeaveGroupWrongCoordinator() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val leaveGroupResult = leaveGroup(groupId, memberId, isCoordinatorForGroup = false) + val leaveGroupResult = leaveGroup(otherGroupId, memberId) assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, leaveGroupResult) } @Test def testLeaveGroupUnknownGroup() { - val leaveGroupResult = leaveGroup(groupId, memberId, isCoordinatorForGroup = true) + val leaveGroupResult = leaveGroup(groupId, memberId) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult) } @@ -663,13 +645,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID val otherMemberId = "consumerId" - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val leaveGroupResult = leaveGroup(groupId, otherMemberId, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val leaveGroupResult = leaveGroup(groupId, otherMemberId) assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult) } @@ -677,14 +658,13 @@ class GroupCoordinatorResponseTest extends JUnitSuite { def testValidLeaveGroup() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID - val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols, - isCoordinatorForGroup = true) + val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols) val assignedMemberId = joinGroupResult.memberId val joinGroupErrorCode = joinGroupResult.errorCode assertEquals(Errors.NONE.code, joinGroupErrorCode) - EasyMock.reset(groupManager) - val leaveGroupResult = leaveGroup(groupId, assignedMemberId, isCoordinatorForGroup = true) + EasyMock.reset(replicaManager) + val leaveGroupResult = leaveGroup(groupId, assignedMemberId) assertEquals(Errors.NONE.code, leaveGroupResult) } @@ -721,29 +701,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { memberId: String, sessionTimeout: Int, protocolType: String, - protocols: List[(String, Array[Byte])], - isCoordinatorForGroup: Boolean): Future[JoinGroupResult] = { + protocols: List[(String, Array[Byte])]): Future[JoinGroupResult] = { val (responseFuture, responseCallback) = setupJoinGroupCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) - EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) - if (!groupsCache.contains(groupId)) { - val groupMetadata = new GroupMetadata(groupId, protocolType) - - - - groupMetadata.add(memberId, new MemberMetadata(memberId, groupId, sessionTimeout, protocols)) - groupsCache.putIfNotExists(groupId, groupMetadata) - - EasyMock.expect(groupManager.addGroup(groupId, protocolType)).andReturn(groupMetadata) - } - EasyMock.expect(groupManager.propagateAssignment(groupsCache.get(groupId), Errors.REBALANCE_IN_PROGRESS.code)) - .andAnswer(propagateRebalanceAnswer) - - EasyMock.replay(groupManager) + EasyMock.replay(replicaManager) consumerCoordinator.handleJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, responseCallback) responseFuture @@ -753,36 +714,22 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def sendSyncGroupLeader(groupId: String, generation: Int, leaderId: String, - assignment: Map[String, Array[Byte]], - isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { + assignment: Map[String, Array[Byte]]): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - val storeGroupAnswer = new IAnswer[Unit] { - override def answer = { - for (member <- groupsCache.get(groupId).allMembers) { - member.assignment = assignment.getOrElse(member.memberId, Array.empty[Byte]) - if (member.awaitingSyncCallback != null) { - member.awaitingSyncCallback.apply(member.assignment, Errors.NONE.code) - member.awaitingSyncCallback = null - } - } - } - } - - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) - if (groupsCache.contains(groupId)) { - val members = groupsCache.get(groupId).allMembers.filter(_.memberId.equals(leaderId)) - if (members.length != 1) { - responseCallback.apply(null, Errors.UNKNOWN_MEMBER_ID.code) - } else { - members.head.awaitingSyncCallback = responseCallback - } - } - - EasyMock.expect(groupManager.storeGroup(groupsCache.get(groupId), assignment)).andAnswer(storeGroupAnswer) - EasyMock.replay(groupManager) + val capturedArgument: Capture[Map[TopicAndPartition, ProducerResponseStatus] => Unit] = EasyMock.newCapture() + + EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(), + EasyMock.anyShort(), + EasyMock.anyBoolean(), + EasyMock.anyObject().asInstanceOf[Map[TopicAndPartition, MessageSet]], + EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { + override def answer = capturedArgument.getValue.apply( + Map(TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId) -> + new ProducerResponseStatus(Errors.NONE.code, 0L) + ) + )}) + EasyMock.replay(replicaManager) consumerCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) responseFuture @@ -790,31 +737,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def sendSyncGroupFollower(groupId: String, generation: Int, - memberId: String, - isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = { + memberId: String): Future[SyncGroupCallbackParams] = { val (responseFuture, responseCallback) = setupSyncGroupCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) - if (groupsCache.contains(groupId)) { - val members = groupsCache.get(groupId).allMembers.filter(_.memberId.equals(memberId)) - if (members.length != 1) { - responseCallback.apply(null, Errors.UNKNOWN_MEMBER_ID.code) - } else { - val member = members.head - if (member.assignment != null) { - responseCallback.apply(member.assignment, Errors.NONE.code) - } else { - member.awaitingSyncCallback = responseCallback - } - } - } else { - responseCallback.apply(null, Errors.UNKNOWN_MEMBER_ID.code) - } - EasyMock.expect(groupManager.propagateAssignment(groupsCache.get(groupId), Errors.REBALANCE_IN_PROGRESS.code)) - .andAnswer(propagateRebalanceAnswer) - EasyMock.replay(groupManager) + EasyMock.replay(replicaManager) consumerCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback) responseFuture @@ -824,9 +750,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite { memberId: String, sessionTimeout: Int, protocolType: String, - protocols: List[(String, Array[Byte])], - isCoordinatorForGroup: Boolean): JoinGroupResult = { - val responseFuture = sendJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, isCoordinatorForGroup) + protocols: List[(String, Array[Byte])]): JoinGroupResult = { + val responseFuture = sendJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols) // should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay Await.result(responseFuture, Duration(sessionTimeout+100, TimeUnit.MILLISECONDS)) } @@ -834,33 +759,25 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def syncGroupFollower(groupId: String, generationId: Int, - memberId: String, - isCoordinatorForGroup: Boolean): SyncGroupCallbackParams = { - val responseFuture = sendSyncGroupFollower(groupId, generationId, memberId, isCoordinatorForGroup) + memberId: String): SyncGroupCallbackParams = { + val responseFuture = sendSyncGroupFollower(groupId, generationId, memberId) Await.result(responseFuture, Duration(DefaultSessionTimeout+100, TimeUnit.MILLISECONDS)) } private def syncGroupLeader(groupId: String, generationId: Int, memberId: String, - assignment: Map[String, Array[Byte]], - isCoordinatorForGroup: Boolean): SyncGroupCallbackParams = { - val responseFuture = sendSyncGroupLeader(groupId, generationId, memberId, assignment, isCoordinatorForGroup) + assignment: Map[String, Array[Byte]]): SyncGroupCallbackParams = { + val responseFuture = sendSyncGroupLeader(groupId, generationId, memberId, assignment) Await.result(responseFuture, Duration(DefaultSessionTimeout+100, TimeUnit.MILLISECONDS)) } private def heartbeat(groupId: String, consumerId: String, - generationId: Int, - isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = { + generationId: Int): HeartbeatCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) - EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) - EasyMock.replay(groupManager) + EasyMock.replay(replicaManager) consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) @@ -873,39 +790,32 @@ class GroupCoordinatorResponseTest extends JUnitSuite { private def commitOffsets(groupId: String, consumerId: String, generationId: Int, - offsets: Map[TopicAndPartition, OffsetAndMetadata], - isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = { + offsets: immutable.Map[TopicAndPartition, OffsetAndMetadata]): CommitOffsetCallbackParams = { val (responseFuture, responseCallback) = setupCommitOffsetsCallback - val storeOffsetAnswer = new IAnswer[Unit] { - override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code)) - } + val capturedArgument: Capture[Map[TopicAndPartition, ProducerResponseStatus] => Unit] = EasyMock.newCapture() - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) - EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) - EasyMock.expect(groupManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback)) - .andAnswer(storeOffsetAnswer) - EasyMock.replay(groupManager) + EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(), + EasyMock.anyShort(), + EasyMock.anyBoolean(), + EasyMock.anyObject().asInstanceOf[Map[TopicAndPartition, MessageSet]], + EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { + override def answer = capturedArgument.getValue.apply( + Map(TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId) -> + new ProducerResponseStatus(Errors.NONE.code, 0L) + ) + )}) + EasyMock.replay(replicaManager) consumerCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) } - private def leaveGroup(groupId: String, consumerId: String, isCoordinatorForGroup: Boolean): LeaveGroupCallbackParams = { + private def leaveGroup(groupId: String, consumerId: String): LeaveGroupCallbackParams = { val (responseFuture, responseCallback) = setupHeartbeatCallback - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLeaderIsLocal(1)).andReturn(isCoordinatorForGroup) - EasyMock.expect(groupManager.partitionFor(groupId)).andReturn(1) - EasyMock.expect(groupManager.partitionLoadingInProgress(1)).andReturn(false) - EasyMock.expect(groupManager.getGroup(groupId)).andReturn(groupsCache.get(groupId)) - EasyMock.expect(groupManager.propagateAssignment(groupsCache.get(groupId), Errors.REBALANCE_IN_PROGRESS.code)) - .andAnswer(propagateRebalanceAnswer) - EasyMock.expect(groupManager.removeGroup(groupId)) - EasyMock.replay(groupManager) + EasyMock.expect(replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId)).andReturn(None) + EasyMock.replay(replicaManager) consumerCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) From 2db31f23d695b90e527e904dce869591514f05e9 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sun, 1 Nov 2015 01:11:47 -0800 Subject: [PATCH 20/23] fix one bug --- .../src/main/scala/kafka/coordinator/GroupMetadataManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 2b0b3accf92b6..a77ab943da7bf 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -224,7 +224,7 @@ class GroupMetadataManager(val brokerId: Int, def propagateAssignment(group: GroupMetadata, errorCode: Short) { - val hasError = errorCode == Errors.NONE.code + val hasError = errorCode != Errors.NONE.code for (member <- group.allMembers) { if (member.awaitingSyncCallback != null) { member.awaitingSyncCallback(if (hasError) Array.empty else member.assignment, errorCode) From 0e0d67fa9f1d6676af77d322f9dc1fed272493e5 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sun, 1 Nov 2015 19:50:46 -0800 Subject: [PATCH 21/23] include client id in member id, remove group in cache upon partition migration --- .../kafka/coordinator/GroupCoordinator.scala | 28 ++++---- .../kafka/coordinator/GroupMetadata.scala | 4 +- .../coordinator/GroupMetadataManager.scala | 64 +++++++++++++------ .../main/scala/kafka/server/KafkaApis.scala | 1 + .../GroupCoordinatorResponseTest.scala | 2 +- .../kafka/coordinator/GroupMetadataTest.scala | 12 ---- 6 files changed, 63 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 7144a0833011c..ea91c07545e15 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -106,6 +106,7 @@ class GroupCoordinator(val brokerId: Int, def handleJoinGroup(groupId: String, memberId: String, + clientId: String, sessionTimeoutMs: Int, protocolType: String, protocols: List[(String, Array[Byte])], @@ -131,16 +132,17 @@ class GroupCoordinator(val brokerId: Int, responseCallback(joinError(memberId, Errors.UNKNOWN_MEMBER_ID.code)) } else { group = groupManager.addGroup(groupId, protocolType) - doJoinGroup(group, memberId, sessionTimeoutMs, protocolType, protocols, responseCallback) + doJoinGroup(group, memberId, clientId, sessionTimeoutMs, protocolType, protocols, responseCallback) } } else { - doJoinGroup(group, memberId, sessionTimeoutMs, protocolType, protocols, responseCallback) + doJoinGroup(group, memberId, clientId, sessionTimeoutMs, protocolType, protocols, responseCallback) } } } private def doJoinGroup(group: GroupMetadata, memberId: String, + clientId: String, sessionTimeoutMs: Int, protocolType: String, protocols: List[(String, Array[Byte])], @@ -164,7 +166,7 @@ class GroupCoordinator(val brokerId: Int, case PreparingRebalance => if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) { - addMemberAndRebalance(sessionTimeoutMs, protocols, group, responseCallback) + addMemberAndRebalance(sessionTimeoutMs, clientId, protocols, group, responseCallback) } else { val member = group.get(memberId) updateMemberAndRebalance(group, member, protocols, responseCallback) @@ -172,7 +174,7 @@ class GroupCoordinator(val brokerId: Int, case AwaitingSync => if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) { - addMemberAndRebalance(sessionTimeoutMs, protocols, group, responseCallback) + addMemberAndRebalance(sessionTimeoutMs, clientId, protocols, group, responseCallback) } else { val member = group.get(memberId) if (member.matches(protocols)) { @@ -199,7 +201,7 @@ class GroupCoordinator(val brokerId: Int, case Stable => if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) { // if the member id is unknown, register the member to the group - addMemberAndRebalance(sessionTimeoutMs, protocols, group, responseCallback) + addMemberAndRebalance(sessionTimeoutMs, clientId, protocols, group, responseCallback) } else { val member = group.get(memberId) if (memberId == group.leaderId || !member.matches(protocols)) { @@ -332,14 +334,14 @@ class GroupCoordinator(val brokerId: Int, } else { val group = groupManager.getGroup(groupId) if (group == null) { - // if the group is marked as dead, it means some other thread has just removed the group - // from the coordinator metadata; this is likely that the group has migrated to some other - // coordinator OR the group is in a transient unstable phase. Let the member retry - // joining without the specified member id, responseCallback(Errors.UNKNOWN_MEMBER_ID.code) } else { group synchronized { if (group.is(Dead)) { + // if the group is marked as dead, it means some other thread has just removed the group + // from the coordinator metadata; this is likely that the group has migrated to some other + // coordinator OR the group is in a transient unstable phase. Let the member retry + // joining without the specified member id, responseCallback(Errors.UNKNOWN_MEMBER_ID.code) } else if (!group.is(Stable)) { responseCallback(Errors.REBALANCE_IN_PROGRESS.code) @@ -455,10 +457,12 @@ class GroupCoordinator(val brokerId: Int, } private def addMemberAndRebalance(sessionTimeoutMs: Int, + clientId: String, protocols: List[(String, Array[Byte])], group: GroupMetadata, callback: JoinCallback) = { - val memberId = group.generateNextMemberId + // use the client-id with a random id suffix as the member-id + val memberId = clientId + "-" + group.generateMemberIdSuffix val member = new MemberMetadata(memberId, group.groupId, sessionTimeoutMs, protocols) member.awaitingJoinCallback = callback group.add(member.memberId, member) @@ -529,10 +533,10 @@ class GroupCoordinator(val brokerId: Int, // TODO: cut the socket connection to the client } + // TODO KAFKA-2720: only remove group in the background thread if (group.isEmpty) { - group.transitionTo(Dead) + groupManager.removeGroup(group) info("Group %s generation %s is dead and removed".format(group.groupId, group.generationId)) - groupManager.removeGroup(group.groupId) } } if (!group.is(Dead)) { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala index 20433d47ce1e1..652a3a4a8d50a 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala @@ -86,7 +86,7 @@ private[coordinator] case object Dead extends GroupState { val state: Byte = 4 } private object GroupMetadata { private val validPreviousStates: Map[GroupState, Set[GroupState]] = - Map(Dead -> Set(PreparingRebalance), + Map(Dead -> Set(Stable, PreparingRebalance, AwaitingSync), AwaitingSync -> Set(PreparingRebalance), Stable -> Set(AwaitingSync), PreparingRebalance -> Set(Stable, AwaitingSync)) @@ -151,7 +151,7 @@ private[coordinator] class GroupMetadata(val groupId: String, val protocolType: } // TODO: decide if ids should be predictable or random - def generateNextMemberId = UUID.randomUUID().toString + def generateMemberIdSuffix = UUID.randomUUID().toString def canRebalance = state == Stable || state == AwaitingSync diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index a77ab943da7bf..46b3da694cd83 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -58,16 +58,16 @@ class GroupMetadataManager(val brokerId: Int, /* group metadata cache */ private val groupsCache = new Pool[String, GroupMetadata] - /* partitions of consumer groups that are being loaded */ + /* partitions of consumer groups that are being loaded, its lock should be always called BEFORE the group lock if needed */ private val loadingPartitions: mutable.Set[Int] = mutable.Set() /* partitions of consumer groups that are assigned, using the same loading partition lock */ private val ownedPartitions: mutable.Set[Int] = mutable.Set() - /* lock for expiring stale offsets */ + /* lock for expiring stale offsets, it should be always called BEFORE the group lock if needed */ private val offsetExpireLock = new ReentrantReadWriteLock() - /* lock for removing offsets of a range partition */ + /* lock for removing offsets of a range partition, it should be always called BEFORE the group lock if needed */ private val offsetRemoveLock = new ReentrantReadWriteLock() /* shutting down flag */ @@ -123,32 +123,35 @@ class GroupMetadataManager(val brokerId: Int, /** * Remove all metadata associated with the group, note this function needs to be * called inside the group lock - * @param groupId the groupId of the group we are removing + * @param group */ - def removeGroup(groupId: String) { - if (groupsCache.remove(groupId) == null) - throw new IllegalArgumentException("Cannot remove non-existing group") + def removeGroup(group: GroupMetadata) { + // first mark the group as dead + group.transitionTo(Dead) + + if (groupsCache.remove(group.groupId) != group) + throw new IllegalArgumentException("Cannot remove group " + group.groupId + " since it has been replaced.") // Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say, // if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and // retry removing this group. - val groupPartition = partitionFor(groupId) - val tomstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(groupId)) + val groupPartition = partitionFor(group.groupId) + val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId)) val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) partitionOpt.foreach { partition => val appendPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) - trace("Marking group %s as deleted.".format(groupId)) + trace("Marking group %s as deleted.".format(group.groupId)) try { - // do not need to require acks since even if the tombsone is lost, + // do not need to require acks since even if the tombstone is lost, // it will be appended again by the new leader - // TODO: have periodic purging instead of immediate removal of groups - partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, tomstone)) + // TODO KAFKA-2720: periodic purging instead of immediate removal of groups + partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, tombstone)) } catch { case t: Throwable => - error("Failed to mark group %s as deleted in %s.".format(groupId, appendPartition), t) + error("Failed to mark group %s as deleted in %s.".format(group.groupId, appendPartition), t) // ignore and continue } } @@ -447,22 +450,41 @@ class GroupMetadataManager(val brokerId: Int, * @param offsetsPartition Groups belonging to this partition of the offsets topic will be deleted from the cache. */ def removeGroupsForPartition(offsetsPartition: Int) { - var numRemoved = 0 + var numOffsetsRemoved = 0 inWriteLock(offsetRemoveLock) { offsetsCache.keys.foreach { key => if (partitionFor(key.group) == offsetsPartition) { offsetsCache.remove(key) - numRemoved += 1 + numOffsetsRemoved += 1 } } - - // TODO: we may also remove the group metadata cache here } - if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition." - .format(numRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) + if (numOffsetsRemoved > 0) info("Removed %d cached offsets for %s on follower transition." + .format(numOffsetsRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) + + var numGroupsRemoved = 0 + loadingPartitions synchronized { + // we need to guard the group removal in cache in the loading partition lock + // to prevent coordinator's check-and-get-group race condition + ownedPartitions.remove(offsetsPartition) + + // clear the groups for this partition in the cache + for (group <- groupsCache.values) { + group synchronized { + // mark the group as dead and then remove it from cache + group.transitionTo(Dead) + + if (groupsCache.remove(group.groupId) != group) + throw new IllegalArgumentException("Cannot remove group " + group.groupId + " since it has been replaced.") + + numGroupsRemoved += 1 + } + } + } - loadingPartitions synchronized ownedPartitions.remove(offsetsPartition) + if (numGroupsRemoved > 0) info("Removed %d cached groups for %s on follower transition." + .format(numGroupsRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) } /** diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index e65bfc73ff169..35c5956559425 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -739,6 +739,7 @@ class KafkaApis(val requestChannel: RequestChannel, coordinator.handleJoinGroup( joinGroupRequest.groupId(), joinGroupRequest.memberId(), + request.header.clientId(), joinGroupRequest.sessionTimeout(), joinGroupRequest.protocolType(), protocols, diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 2acb871355604..5e6bd036fad6d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -706,7 +706,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.replay(replicaManager) - consumerCoordinator.handleJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, responseCallback) + consumerCoordinator.handleJoinGroup(groupId, memberId, "clientId", sessionTimeout, protocolType, protocols, responseCallback) responseFuture } diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala index 0f3e74815af92..021aea6af17eb 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala @@ -96,11 +96,6 @@ class GroupMetadataTest extends JUnitSuite { group.transitionTo(AwaitingSync) } - @Test(expected = classOf[IllegalStateException]) - def testStableToDeadIllegalTransition() { - group.transitionTo(Dead) - } - @Test(expected = classOf[IllegalStateException]) def testPreparingRebalanceToPreparingRebalanceIllegalTransition() { group.transitionTo(PreparingRebalance) @@ -120,13 +115,6 @@ class GroupMetadataTest extends JUnitSuite { group.transitionTo(AwaitingSync) } - @Test(expected = classOf[IllegalStateException]) - def testAwaitingSyncToDeadIllegalTransition() { - group.transitionTo(PreparingRebalance) - group.transitionTo(AwaitingSync) - group.transitionTo(Dead) - } - @Test(expected = classOf[IllegalStateException]) def testDeadToDeadIllegalTransition() { group.transitionTo(PreparingRebalance) From 8ca6ed2370f13f9db20eb23776ab862fe5ec843b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 2 Nov 2015 11:57:41 -0800 Subject: [PATCH 22/23] address Jun's comments --- .../kafka/coordinator/GroupCoordinator.scala | 4 ++-- .../coordinator/GroupMetadataManager.scala | 22 +++++++++---------- .../kafka/coordinator/MemberMetadata.scala | 7 ++++-- 3 files changed, 17 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index ea91c07545e15..5408c9ec5a9bc 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -588,9 +588,9 @@ class GroupCoordinator(val brokerId: Int, member.awaitingSyncCallback != null || member.latestHeartbeat + member.sessionTimeoutMs > heartbeatDeadline - private def isCoordinatorForGroup(groupId: String) = groupManager.partitionLeaderIsLocal(groupManager.partitionFor(groupId)) + private def isCoordinatorForGroup(groupId: String) = groupManager.isGroupLocal(groupId) - private def isCoordinatorLoadingInProgress(groupId: String) = groupManager.partitionLoadingInProgress(groupManager.partitionFor(groupId)) + private def isCoordinatorLoadingInProgress(groupId: String) = groupManager.isGroupLoading(groupId) } object GroupCoordinator { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 46b3da694cd83..da295b29aac06 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -91,15 +91,15 @@ class GroupMetadataManager(val brokerId: Int, newGauge("NumGroups", new Gauge[Int] { - def value = offsetsCache.keys.map(_.group).toSet.size + def value = groupsCache.size } ) - def partitionFor(group: String): Int = Utils.abs(group.hashCode) % groupMetadataTopicPartitionCount + def partitionFor(groupId: String): Int = Utils.abs(groupId.hashCode) % groupMetadataTopicPartitionCount - def partitionLeaderIsLocal(partition: Int): Boolean = loadingPartitions synchronized ownedPartitions.contains(partition) + def isGroupLocal(groupId: String): Boolean = loadingPartitions synchronized ownedPartitions.contains(partitionFor(groupId)) - def partitionLoadingInProgress(partition: Int): Boolean = loadingPartitions synchronized loadingPartitions.contains(partition) + def isGroupLoading(groupId: String): Boolean = loadingPartitions synchronized loadingPartitions.contains(partitionFor(groupId)) /** * Get the group associated with the given groupId, or null if not found @@ -170,15 +170,15 @@ class GroupMetadataManager(val brokerId: Int, val groupMetadataMessageSet = Map(groupMetadataPartition -> new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message)) - // set the callback function to insert offsets into cache after log append completed + // set the callback function to insert the created group into cache after log append completed def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { // the append response should only contain the topics partition if (responseStatus.size != 1 || ! responseStatus.contains(groupMetadataPartition)) throw new IllegalStateException("Append status %s should only have one partition %s" .format(responseStatus, groupMetadataPartition)) - // construct the commit response status and insert - // the offset and metadata to cache if the append status has no error + // construct the error status in the propagated assignment response + // in the cache val status = responseStatus(groupMetadataPartition) var responseCode = Errors.NONE.code @@ -213,10 +213,10 @@ class GroupMetadataManager(val brokerId: Int, } // propagate the assignments - propagateAssignment(group, Errors.NONE.code) + propagateAssignment(group, responseCode) } - // call replica manager to append the offset messages + // call replica manager to append the group message replicaManager.appendMessages( config.offsetCommitTimeoutMs.toLong, config.offsetCommitRequiredAcks, @@ -325,14 +325,12 @@ class GroupMetadataManager(val brokerId: Int, def getOffsets(group: String, topicPartitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = { trace("Getting offsets %s for group %s.".format(topicPartitions, group)) - val offsetsPartition = partitionFor(group) - /** * we need to put the leader-is-local check inside the offsetLock to protects against fetching from an empty/cleared * offset cache (i.e., cleared due to a leader->follower transition right after the check and clear the cache). */ inReadLock(offsetRemoveLock) { - if (partitionLeaderIsLocal(offsetsPartition)) { + if (isGroupLocal(group)) { if (topicPartitions.isEmpty) { // 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) => diff --git a/core/src/main/scala/kafka/coordinator/MemberMetadata.scala b/core/src/main/scala/kafka/coordinator/MemberMetadata.scala index 7f7df9ab3b38d..6a762412796bb 100644 --- a/core/src/main/scala/kafka/coordinator/MemberMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/MemberMetadata.scala @@ -49,9 +49,12 @@ private[coordinator] class MemberMetadata(val memberId: String, val sessionTimeoutMs: Int, var supportedProtocols: List[(String, Array[Byte])]) { - var assignment: Array[Byte] = null + // NOTE: we need to add memory barrier to assignment and awaitingSyncCallback + // since they can be accessed in the append callback thread that does not + // hold on the group object lock + @volatile var assignment: Array[Byte] = null var awaitingJoinCallback: JoinGroupResult => Unit = null - var awaitingSyncCallback: (Array[Byte], Short) => Unit = null + @volatile var awaitingSyncCallback: (Array[Byte], Short) => Unit = null var latestHeartbeat: Long = -1 var isLeaving: Boolean = false From a968e3b7950ab89843e31674d38d5d803304ae13 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 2 Nov 2015 21:50:09 -0800 Subject: [PATCH 23/23] address github comments round two --- ...java => GroupLoadInProgressException.java} | 14 ++-- .../apache/kafka/common/protocol/Errors.java | 2 +- .../kafka/coordinator/GroupCoordinator.scala | 2 +- .../coordinator/GroupMetadataManager.scala | 79 +++++++++++-------- 4 files changed, 55 insertions(+), 42 deletions(-) rename clients/src/main/java/org/apache/kafka/common/errors/{OffsetLoadInProgressException.java => GroupLoadInProgressException.java} (67%) diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java similarity index 67% rename from clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java rename to clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java index 016506e9d2463..17e205f018cc3 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/OffsetLoadInProgressException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/GroupLoadInProgressException.java @@ -14,26 +14,26 @@ package org.apache.kafka.common.errors; /** - * The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change - * for that offsets topic partition). + * The broker returns this error code for any coordiantor request if it is still loading the metadata (after a leader change + * for that offsets topic partition) for this group. */ -public class OffsetLoadInProgressException extends RetriableException { +public class GroupLoadInProgressException extends RetriableException { private static final long serialVersionUID = 1L; - public OffsetLoadInProgressException() { + public GroupLoadInProgressException() { super(); } - public OffsetLoadInProgressException(String message) { + public GroupLoadInProgressException(String message) { super(message); } - public OffsetLoadInProgressException(String message, Throwable cause) { + public GroupLoadInProgressException(String message, Throwable cause) { super(message, cause); } - public OffsetLoadInProgressException(Throwable cause) { + public GroupLoadInProgressException(Throwable cause) { super(cause); } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index cf537addc7c65..2c9cb209b9274 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -59,7 +59,7 @@ public enum Errors { NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), GROUP_LOAD_IN_PROGRESS(14, - new OffsetLoadInProgressException("The coordinator is loading and hence can't process requests for this group.")), + new GroupLoadInProgressException("The coordinator is loading and hence can't process requests for this group.")), GROUP_COORDINATOR_NOT_AVAILABLE(15, new GroupCoordinatorNotAvailableException("The group coordinator is not available.")), NOT_COORDINATOR_FOR_GROUP(16, diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 5408c9ec5a9bc..97ce22be640e3 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.requests.JoinGroupRequest import scala.collection.{Map, Seq, immutable} case class GroupConfig(groupMinSessionTimeoutMs: Int, - groupMaxSessionTimeoutMs: Int) + groupMaxSessionTimeoutMs: Int) case class JoinGroupResult(members: Map[String, Array[Byte]], memberId: String, diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index da295b29aac06..81ed54850d90f 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -117,7 +117,7 @@ class GroupMetadataManager(val brokerId: Int, private def addGroup(groupId: String, group: GroupMetadata): GroupMetadata = { groupsCache.putIfNotExists(groupId, group) - group + groupsCache.get(groupId) } /** @@ -325,30 +325,24 @@ class GroupMetadataManager(val brokerId: Int, def getOffsets(group: String, topicPartitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = { trace("Getting offsets %s for group %s.".format(topicPartitions, group)) - /** - * we need to put the leader-is-local check inside the offsetLock to protects against fetching from an empty/cleared - * offset cache (i.e., cleared due to a leader->follower transition right after the check and clear the cache). - */ - inReadLock(offsetRemoveLock) { - if (isGroupLocal(group)) { - if (topicPartitions.isEmpty) { - // 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 - } + if (isGroupLocal(group)) { + if (topicPartitions.isEmpty) { + // 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 { - debug("Could not fetch offsets for group %s (not offset coordinator).".format(group)) topicPartitions.map { topicAndPartition => val groupTopicPartition = GroupTopicPartition(group, topicAndPartition) - (groupTopicPartition.topicPartition, OffsetMetadataAndError.NotCoordinatorForGroup) + (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.NotCoordinatorForGroup) + }.toMap } } @@ -449,23 +443,26 @@ class GroupMetadataManager(val brokerId: Int, */ def removeGroupsForPartition(offsetsPartition: Int) { var numOffsetsRemoved = 0 - inWriteLock(offsetRemoveLock) { + var numGroupsRemoved = 0 + + loadingPartitions synchronized { + // we need to guard the group removal in cache in the loading partition lock + // to prevent coordinator's check-and-get-group race condition + ownedPartitions.remove(offsetsPartition) + + // clear the offsets for this partition in the cache + + /** + * NOTE: we need to put this in the loading partition lock as well to prevent race condition of the leader-is-local check + * in getOffsets to protects against fetching from an empty/cleared offset cache (i.e., cleared due to a leader->follower + * transition right after the check and clear the cache), causing offset fetch return empty offsets with NONE error code + */ offsetsCache.keys.foreach { key => if (partitionFor(key.group) == offsetsPartition) { offsetsCache.remove(key) numOffsetsRemoved += 1 } } - } - - if (numOffsetsRemoved > 0) info("Removed %d cached offsets for %s on follower transition." - .format(numOffsetsRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) - - var numGroupsRemoved = 0 - loadingPartitions synchronized { - // we need to guard the group removal in cache in the loading partition lock - // to prevent coordinator's check-and-get-group race condition - ownedPartitions.remove(offsetsPartition) // clear the groups for this partition in the cache for (group <- groupsCache.values) { @@ -481,6 +478,9 @@ class GroupMetadataManager(val brokerId: Int, } } + if (numOffsetsRemoved > 0) info("Removed %d cached offsets for %s on follower transition." + .format(numOffsetsRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) + if (numGroupsRemoved > 0) info("Removed %d cached groups for %s on follower transition." .format(numGroupsRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition))) } @@ -610,7 +610,21 @@ class GroupMetadataManager(val brokerId: Int, } } - +/** + * Messages stored for the group topic has versions for both the key and value fields. Key + * version is used to indicate the type of the message (also to differentiate different types + * of messages from being compacted together if they have the same field values); and value + * version is used to evolve the messages within their data types: + * + * key version 0: group consumption offset + * -> value version 0: [offset, metadata, timestamp] + * + * key version 1: group consumption offset + * -> value version 1: [offset, metadata, commit_timestamp, expire_timestamp] + * + * key version 2: group metadata + * -> value version 0: [protocol_type, generation, protocol, leader, members] + */ object GroupMetadataManager { private val CURRENT_OFFSET_KEY_SCHEMA_VERSION = 1.toShort @@ -630,7 +644,6 @@ object GroupMetadataManager { private val OFFSET_VALUE_METADATA_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") private val OFFSET_VALUE_TIMESTAMP_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") - private val OFFSET_COMMIT_VALUE_SCHEMA_V1 = new Schema(new Field("offset", INT64), new Field("metadata", STRING, "Associated metadata.", ""), new Field("commit_timestamp", INT64),