From 2bcafb4ba2be54e195815ccce7d7bc042315f405 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 8 Aug 2014 17:12:51 -0700 Subject: [PATCH 01/33] step 1 --- .../kafka/server/DelayedOperationPurgatory.scala | 285 +++++++++++++++++++++ .../server/FetchDelayedOperationPurgatory.scala | 69 +++++ .../scala/kafka/server/FetchRequestPurgatory.scala | 69 ----- core/src/main/scala/kafka/server/KafkaApis.scala | 4 +- .../server/ProducerDelayedOperationPurgatory.scala | 69 +++++ .../kafka/server/ProducerRequestPurgatory.scala | 69 ----- .../main/scala/kafka/server/ReplicaManager.scala | 8 +- core/src/main/scala/kafka/utils/DelayedItem.scala | 6 +- .../server/DelayedOperationPurgatoryTest.scala | 131 ++++++++++ .../unit/kafka/server/RequestPurgatoryTest.scala | 123 --------- 10 files changed, 563 insertions(+), 270 deletions(-) create mode 100644 core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala create mode 100644 core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala delete mode 100644 core/src/main/scala/kafka/server/FetchRequestPurgatory.scala create mode 100644 core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala delete mode 100644 core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala create mode 100644 core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala delete mode 100644 core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala diff --git a/core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala b/core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala new file mode 100644 index 0000000..3998b6d --- /dev/null +++ b/core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala @@ -0,0 +1,285 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.network._ +import kafka.utils._ +import kafka.metrics.KafkaMetricsGroup + +import java.util +import java.util.concurrent._ +import java.util.concurrent.atomic._ +import scala.collection._ + +import com.yammer.metrics.core.Gauge + + +/** + * An operation whose processing needs to be delayed for at most the given delayMs; + * upon complete, the given callback function will be triggered. For example a delayed + * message append operation could be waiting for specified number of acks; or a delayed + * message fetch operation could be waiting for a given number of bytes to accumulate. + */ +abstract class DelayedOperation(delayMs: Long, onComplete: Boolean => Unit) extends DelayedItem(delayMs) { + val completed = new AtomicBoolean(false) + + /* + * Check if the delayed operation is already completed + * + * Note that concurrent threads can check if an operation can be completed or not, + * but only the first thread will succeed in completing the operation + */ + def tryComplete(): Boolean = completed.compareAndSet(false, true) + + /* + * When delayMs has elapsed, expire the delayed operation + */ + def onExpired() = onComplete(false) +} + +/** + * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. + * + */ +abstract class DelayedOperationPurgatory[T <: DelayedOperation](brokerId: Int = 0, purgeInterval: Int = 1000) + extends Logging with KafkaMetricsGroup { + + /* a list of requests watching each key */ + private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) + + /* the number of requests being watched, duplicates added on different watchers are also counted */ + private val watched = new AtomicInteger(0) + + /* background thread expiring requests that have been waiting too long */ + private val expirationReaper = new ExpiredOperationReaper + + newGauge( + "PurgatorySize", + new Gauge[Int] { + def value = size + } + ) + + newGauge( + "NumDelayedOperations", + new Gauge[Int] { + def value = expirationReaper.numOperations + } + ) + + expirationThread.start() + + /** + * Check if the operation can be completed, if not watch it based on the given watch keys + * + * Note that a delayed operation can be watched on multiple keys, and hence due to concurrency may be + * found completed when trying to watch it on some later keys. In this case the operation is still + * treated as completed and hence no longer watched although it is still in the watch lists of + * the earlier keys. Those already watched elements will be later purged by the expire reaper. + * + * @param operation the delayed operation to be checked + * @param watchKeys keys for bookkeeping the operation + * @return true iff the delayed operations can be completed + */ + def tryCompleteElseWatch(operation: DelayedOperation, watchKeys: Seq[Any]): Boolean = { + for(key <- watchKeys) { + val watchers = watchersFor(key) + // if the operation is found completed, stop adding it to any further + // lists and return true immediately + if(!watchers.checkAndMaybeAdd(operation)) { + return true + } + } + + // if it is indeed watched, add to the expire queue also + watched.getAndIncrement() + expirationReaper.enqueue(operation) + + false + + } + + /** + * Return a list of completed operations with the given watch key. + */ + def getCompleted(key: Any): Seq[T] = { + val watchers = watchersForKey.get(key) + if(watchers == null) + Seq.empty + else + watchers.collectCompletedOperations() + } + + /* + * Return the watch list of the given key + */ + private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) + + /* + * Return the size of the purgatory, which is size of watch lists plus the size of the expire reaper. + * Since an operation may still be in the watch lists even when it has been completed, this number + * may be larger than the number of real operations watched + */ + protected def size() = watchersForKey.values.map(_.numRequests).sum + expirationReaper.numOperations + + /** + * Shutdown the expire reaper thread + */ + def shutdown() { + expirationReaper.shutdown() + } + + /** + * A linked list of watched delayed operations based on some key + */ + private class Watchers { + private val requests = new util.ArrayList[T] + + // potentially add the element to watch if it is not satisfied yet + def checkAndMaybeAdd(t: T): Boolean = { + synchronized { + // if it is already satisfied, return false + if (t.completed.get()) + return false + // if the operation can be completed, return false; otherwise add to watch list + if(t.tryComplete()) { + return false + } else { + requests.add(t) + return true + } + } + } + + // traverse the list and purge satisfied elements + def purgeSatisfied(): Int = { + synchronized { + val iter = requests.iterator() + var purged = 0 + while (iter.hasNext) { + val curr = iter.next + if(curr.completed.get()) { + iter.remove() + purged += 1 + } + } + purged + } + } + + // traverse the list and try to satisfy watched elements + def collectCompletedOperations(): Seq[T] = { + val response = new mutable.ArrayBuffer[T] + synchronized { + val iter = requests.iterator() + while(iter.hasNext) { + val curr = iter.next + if (curr.completed.get()) { + // another thread has completed this request, just remove it + iter.remove() + } else { + val completed = curr.tryComplete() + if(completed) { + iter.remove() + watched.getAndDecrement() + response += curr + expirationReaper.satisfyRequest() + } + } + } + } + response + } + } + + /** + * A background reaper to expire delayed operations that have timed out + */ + private class ExpiredOperationReaper extends ShutdownableThread( + "ExpirationReaper-%d".format(brokerId), + false) { + + /* The queue storing all delayed operations */ + private val delayed = new DelayQueue[T] + + /* + * Return the number of delayed operations kept by the reaper + */ + def numOperations = delayed.size() + + /* + * Add a operation to be expired + */ + def enqueue(t: T) { + delayed.add(t) + unsatisfied.incrementAndGet() + } + + /** + * Get the next expired event + */ + private def pollExpired(): T = { + while (true) { + val curr = delayed.poll(200L, TimeUnit.MILLISECONDS) + if (curr == null) + return null.asInstanceOf[T] + // try set the operation failed (and hence completed), if succeed return it; + // otherwise try to get the next expired operation since this one has been completed by others + if (curr.completed.compareAndSet(false, true)) { + return curr + } + } + throw new RuntimeException("This should not happen") + } + + /** + * Delete all satisfied events from the delay queue and the watcher lists + */ + private def purgeSatisfied(): Int = { + var purged = 0 + + // purge the delayed queue + val iter = delayed.iterator() + while (iter.hasNext) { + val curr = iter.next() + if (curr.completed.get()) { + iter.remove() + purged += 1 + } + } + + purged + } + + + override def doWork() { + val curr = pollExpired() + if (curr != null) { + curr.onExpired() + } + if (size >= purgeInterval) { // see if we need to force a full purge + debug("Beginning purgatory purge") + val purged = purgeSatisfied() + debug("Purged %d operations from delay queue.".format(purged)) + val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum + debug("Purged %d operations from watch lists.".format(numPurgedFromWatchers)) + } + } + } + +} diff --git a/core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala b/core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala new file mode 100644 index 0000000..71c5920 --- /dev/null +++ b/core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.metrics.KafkaMetricsGroup +import kafka.network.RequestChannel +import kafka.api.FetchResponseSend + +import java.util.concurrent.TimeUnit + +/** + * The purgatory holding delayed fetch requests + */ +class FetchDelayedOperationPurgatory(replicaManager: ReplicaManager, requestChannel: RequestChannel) + extends DelayedOperationPurgatory[DelayedFetch](replicaManager.config.brokerId, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) { + this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId) + + private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup { + private val metricPrefix = if (forFollower) "Follower" else "Consumer" + + val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) + } + + private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) + private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) + + private def recordDelayedFetchExpired(forFollower: Boolean) { + val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics + else aggregateNonFollowerFetchRequestMetrics + + metrics.expiredRequestMeter.mark() + } + + /** + * Check if a specified delayed fetch request is satisfied + */ + def checkSatisfied(delayedFetch: DelayedFetch): Boolean = delayedFetch.isSatisfied(replicaManager) + + /** + * When a delayed fetch request expires just answer it with whatever data is present + */ + def expire(delayedFetch: DelayedFetch) { + debug("Expiring fetch request %s.".format(delayedFetch.fetch)) + val fromFollower = delayedFetch.fetch.isFromFollower + recordDelayedFetchExpired(fromFollower) + respond(delayedFetch) + } + + // TODO: purgatory should not be responsible for sending back the responses + def respond(delayedFetch: DelayedFetch) { + val response = delayedFetch.respond(replicaManager) + requestChannel.sendResponse(new RequestChannel.Response(delayedFetch.request, new FetchResponseSend(response))) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala b/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala deleted file mode 100644 index ed13188..0000000 --- a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala +++ /dev/null @@ -1,69 +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 kafka.metrics.KafkaMetricsGroup -import kafka.network.RequestChannel -import kafka.api.FetchResponseSend - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed fetch requests - */ -class FetchRequestPurgatory(replicaManager: ReplicaManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedFetch](replicaManager.config.brokerId, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) { - this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup { - private val metricPrefix = if (forFollower) "Follower" else "Consumer" - - val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) - private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) - - private def recordDelayedFetchExpired(forFollower: Boolean) { - val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics - else aggregateNonFollowerFetchRequestMetrics - - metrics.expiredRequestMeter.mark() - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedFetch: DelayedFetch): Boolean = delayedFetch.isSatisfied(replicaManager) - - /** - * When a delayed fetch request expires just answer it with whatever data is present - */ - def expire(delayedFetch: DelayedFetch) { - debug("Expiring fetch request %s.".format(delayedFetch.fetch)) - val fromFollower = delayedFetch.fetch.isFromFollower - recordDelayedFetchExpired(fromFollower) - respond(delayedFetch) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedFetch: DelayedFetch) { - val response = delayedFetch.respond(replicaManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedFetch.request, new FetchResponseSend(response))) - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 85498b4..5bee8f8 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -42,8 +42,8 @@ class KafkaApis(val requestChannel: RequestChannel, val config: KafkaConfig, val controller: KafkaController) extends Logging { - val producerRequestPurgatory = new ProducerRequestPurgatory(replicaManager, offsetManager, requestChannel) - val fetchRequestPurgatory = new FetchRequestPurgatory(replicaManager, requestChannel) + val producerRequestPurgatory = new ProducerDelayedOperationPurgatory(replicaManager, offsetManager, requestChannel) + val fetchRequestPurgatory = new FetchDelayedOperationPurgatory(replicaManager, requestChannel) // TODO: the following line will be removed in 0.9 replicaManager.initWithRequestPurgatory(producerRequestPurgatory, fetchRequestPurgatory) var metadataCache = new MetadataCache diff --git a/core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala b/core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala new file mode 100644 index 0000000..4b950e1 --- /dev/null +++ b/core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.Pool +import kafka.network.{BoundedByteBufferSend, RequestChannel} + +import java.util.concurrent.TimeUnit + +/** + * The purgatory holding delayed producer requests + */ +class ProducerDelayedOperationPurgatory(replicaManager: ReplicaManager, offsetManager: OffsetManager, requestChannel: RequestChannel) + extends DelayedOperationPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) { + this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId) + + private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup { + val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) + } + + private val producerRequestMetricsForKey = { + val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") + new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory)) + } + + private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics + + private def recordDelayedProducerKeyExpired(key: DelayedRequestKey) { + val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) + List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) + } + + /** + * Check if a specified delayed fetch request is satisfied + */ + def checkSatisfied(delayedProduce: DelayedProduce) = delayedProduce.isSatisfied(replicaManager) + + /** + * When a delayed produce request expires answer it with possible time out error codes + */ + def expire(delayedProduce: DelayedProduce) { + debug("Expiring produce request %s.".format(delayedProduce.produce)) + for ((topicPartition, responseStatus) <- delayedProduce.partitionStatus if responseStatus.acksPending) + recordDelayedProducerKeyExpired(new TopicPartitionRequestKey(topicPartition)) + respond(delayedProduce) + } + + // TODO: purgatory should not be responsible for sending back the responses + def respond(delayedProduce: DelayedProduce) { + val response = delayedProduce.respond(offsetManager) + requestChannel.sendResponse(new RequestChannel.Response(delayedProduce.request, new BoundedByteBufferSend(response))) + } +} diff --git a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala b/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala deleted file mode 100644 index d4a7d4a..0000000 --- a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala +++ /dev/null @@ -1,69 +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 kafka.metrics.KafkaMetricsGroup -import kafka.utils.Pool -import kafka.network.{BoundedByteBufferSend, RequestChannel} - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed producer requests - */ -class ProducerRequestPurgatory(replicaManager: ReplicaManager, offsetManager: OffsetManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) { - this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup { - val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val producerRequestMetricsForKey = { - val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") - new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory)) - } - - private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics - - private def recordDelayedProducerKeyExpired(key: DelayedRequestKey) { - val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) - List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedProduce: DelayedProduce) = delayedProduce.isSatisfied(replicaManager) - - /** - * When a delayed produce request expires answer it with possible time out error codes - */ - def expire(delayedProduce: DelayedProduce) { - debug("Expiring produce request %s.".format(delayedProduce.produce)) - for ((topicPartition, responseStatus) <- delayedProduce.partitionStatus if responseStatus.acksPending) - recordDelayedProducerKeyExpired(new TopicPartitionRequestKey(topicPartition)) - respond(delayedProduce) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedProduce: DelayedProduce) { - val response = delayedProduce.respond(offsetManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedProduce.request, new BoundedByteBufferSend(response))) - } -} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 78b7514..9eb8d5d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -46,7 +46,7 @@ object ReplicaManager { case class PartitionDataAndOffset(data: FetchResponsePartitionData, offset: LogOffsetMetadata) -class ReplicaManager(val config: KafkaConfig, +class ReplicaManager(config: KafkaConfig, time: Time, val zkClient: ZkClient, scheduler: Scheduler, @@ -64,8 +64,8 @@ class ReplicaManager(val config: KafkaConfig, this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger - var producerRequestPurgatory: ProducerRequestPurgatory = null - var fetchRequestPurgatory: FetchRequestPurgatory = null + var producerRequestPurgatory: ProducerDelayedOperationPurgatory = null + var fetchRequestPurgatory: FetchDelayedOperationPurgatory = null newGauge( "LeaderCount", @@ -105,7 +105,7 @@ class ReplicaManager(val config: KafkaConfig, * TODO: will be removed in 0.9 where we refactor server structure */ - def initWithRequestPurgatory(producerRequestPurgatory: ProducerRequestPurgatory, fetchRequestPurgatory: FetchRequestPurgatory) { + def initWithRequestPurgatory(producerRequestPurgatory: ProducerDelayedOperationPurgatory, fetchRequestPurgatory: FetchDelayedOperationPurgatory) { this.producerRequestPurgatory = producerRequestPurgatory this.fetchRequestPurgatory = fetchRequestPurgatory } diff --git a/core/src/main/scala/kafka/utils/DelayedItem.scala b/core/src/main/scala/kafka/utils/DelayedItem.scala index d727649..3d7df84 100644 --- a/core/src/main/scala/kafka/utils/DelayedItem.scala +++ b/core/src/main/scala/kafka/utils/DelayedItem.scala @@ -20,7 +20,7 @@ package kafka.utils import java.util.concurrent._ import scala.math._ -class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed with Logging { +class DelayedItem(delay: Long, unit: TimeUnit) extends Delayed with Logging { val createdMs = SystemTime.milliseconds val delayMs = { @@ -29,8 +29,8 @@ class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed w else given } - def this(item: T, delayMs: Long) = - this(item, delayMs, TimeUnit.MILLISECONDS) + def this(delayMs: Long) = + this(delayMs, TimeUnit.MILLISECONDS) /** * The remaining delay time diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala new file mode 100644 index 0000000..ff332d3 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala @@ -0,0 +1,131 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 scala.collection._ +import org.junit.Test +import junit.framework.Assert._ +import kafka.message._ +import kafka.api._ +import kafka.utils.TestUtils +import org.scalatest.junit.JUnit3Suite + + +class DelayedOperationPurgatoryTest extends JUnit3Suite { + + val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes))) + val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes))) + var purgatory: MockDelayedOperationPurgatory = null + + override def setUp() { + super.setUp() +<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala + purgatory = new MockRequestPurgatory(5) +======= + purgatory = new MockDelayedOperationPurgatory() +>>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala + } + + override def tearDown() { + purgatory.shutdown() + super.tearDown() + } + + @Test + def testRequestSatisfaction() { + val r1 = new DelayedRequest(Array("test1"), null, 100000L) + val r2 = new DelayedRequest(Array("test2"), null, 100000L) + assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.update("test1").size) + assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) + assertEquals("Still nothing satisfied", 0, purgatory.update("test1").size) + assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) + assertEquals("Still nothing satisfied", 0, purgatory.update("test2").size) + purgatory.satisfied += r1 + assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1")) + assertEquals("Nothing satisfied", 0, purgatory.update("test1").size) + purgatory.satisfied += r2 + assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2")) + assertEquals("Nothing satisfied", 0, purgatory.update("test2").size) + } + + @Test + def testRequestExpiry() { + val expiration = 20L + val r1 = new DelayedRequest(Array("test1"), null, expiration) + val r2 = new DelayedRequest(Array("test1"), null, 200000L) + val start = System.currentTimeMillis + assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) + assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) + purgatory.awaitExpiration(r1) + val elapsed = System.currentTimeMillis - start + assertTrue("r1 expired", purgatory.expired.contains(r1)) + assertTrue("r2 hasn't expired", !purgatory.expired.contains(r2)) + assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) + } + + @Test + def testRequestPurge() { + val r1 = new DelayedRequest(Array("test1"), null, 100000L) + val r12 = new DelayedRequest(Array("test1", "test2"), null, 100000L) + val r23 = new DelayedRequest(Array("test2", "test3"), null, 100000L) + purgatory.checkAndMaybeWatch(r1) + purgatory.checkAndMaybeWatch(r12) + purgatory.checkAndMaybeWatch(r23) + + assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) + assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) + + // satisfy one of the requests, it should then be purged from the watch list with purge interval 5 + r12.satisfied.set(true) + TestUtils.waitUntilTrue(() => purgatory.watched() == 3, + "Purgatory should have 3 watched elements instead of " + + purgatory.watched(), 1000L) + TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, + "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) + + // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 + purgatory.checkAndMaybeWatch(r1) + purgatory.checkAndMaybeWatch(r1) + + TestUtils.waitUntilTrue(() => purgatory.watched() == 5, + "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) + TestUtils.waitUntilTrue(() => purgatory.delayed() == 4, + "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) + } + +<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala + class MockRequestPurgatory(purge: Int) extends RequestPurgatory[DelayedRequest](purgeInterval = purge) { +======= + class MockDelayedOperationPurgatory extends DelayedOperationPurgatory[DelayedRequest] { +>>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala + val satisfied = mutable.Set[DelayedRequest]() + val expired = mutable.Set[DelayedRequest]() + def awaitExpiration(delayed: DelayedRequest) = { + delayed synchronized { + delayed.wait() + } + } + def checkSatisfied(delayed: DelayedRequest): Boolean = satisfied.contains(delayed) + def expire(delayed: DelayedRequest) { + expired += delayed + delayed synchronized { + delayed.notify() + } + } + } + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala deleted file mode 100644 index a577f4a..0000000 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ /dev/null @@ -1,123 +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 scala.collection._ -import org.junit.Test -import junit.framework.Assert._ -import kafka.message._ -import kafka.api._ -import kafka.utils.TestUtils -import org.scalatest.junit.JUnit3Suite - - -class RequestPurgatoryTest extends JUnit3Suite { - - val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes))) - val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes))) - var purgatory: MockRequestPurgatory = null - - override def setUp() { - super.setUp() - purgatory = new MockRequestPurgatory(5) - } - - override def tearDown() { - purgatory.shutdown() - super.tearDown() - } - - @Test - def testRequestSatisfaction() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r2 = new DelayedRequest(Array("test2"), null, 100000L) - assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.update("test1").size) - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test1").size) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test2").size) - purgatory.satisfied += r1 - assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1")) - assertEquals("Nothing satisfied", 0, purgatory.update("test1").size) - purgatory.satisfied += r2 - assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2")) - assertEquals("Nothing satisfied", 0, purgatory.update("test2").size) - } - - @Test - def testRequestExpiry() { - val expiration = 20L - val r1 = new DelayedRequest(Array("test1"), null, expiration) - val r2 = new DelayedRequest(Array("test1"), null, 200000L) - val start = System.currentTimeMillis - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - purgatory.awaitExpiration(r1) - val elapsed = System.currentTimeMillis - start - assertTrue("r1 expired", purgatory.expired.contains(r1)) - assertTrue("r2 hasn't expired", !purgatory.expired.contains(r2)) - assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) - } - - @Test - def testRequestPurge() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r12 = new DelayedRequest(Array("test1", "test2"), null, 100000L) - val r23 = new DelayedRequest(Array("test2", "test3"), null, 100000L) - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r12) - purgatory.checkAndMaybeWatch(r23) - - assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) - assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) - - // satisfy one of the requests, it should then be purged from the watch list with purge interval 5 - r12.satisfied.set(true) - TestUtils.waitUntilTrue(() => purgatory.watched() == 3, - "Purgatory should have 3 watched elements instead of " + + purgatory.watched(), 1000L) - TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, - "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) - - // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r1) - - TestUtils.waitUntilTrue(() => purgatory.watched() == 5, - "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) - TestUtils.waitUntilTrue(() => purgatory.delayed() == 4, - "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) - } - - class MockRequestPurgatory(purge: Int) extends RequestPurgatory[DelayedRequest](purgeInterval = purge) { - val satisfied = mutable.Set[DelayedRequest]() - val expired = mutable.Set[DelayedRequest]() - def awaitExpiration(delayed: DelayedRequest) = { - delayed synchronized { - delayed.wait() - } - } - def checkSatisfied(delayed: DelayedRequest): Boolean = satisfied.contains(delayed) - def expire(delayed: DelayedRequest) { - expired += delayed - delayed synchronized { - delayed.notify() - } - } - } - -} \ No newline at end of file -- 1.7.12.4 From 8cb68567f16d9226257d80a717f1ab86ca003195 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 12 Aug 2014 09:49:17 -0700 Subject: [PATCH 02/33] change name back --- .../kafka/server/DelayedOperationPurgatory.scala | 285 --------------------- .../server/FetchDelayedOperationPurgatory.scala | 69 ----- .../scala/kafka/server/FetchRequestPurgatory.scala | 69 +++++ core/src/main/scala/kafka/server/KafkaApis.scala | 4 +- .../server/ProducerDelayedOperationPurgatory.scala | 69 ----- .../kafka/server/ProducerRequestPurgatory.scala | 69 +++++ .../main/scala/kafka/server/ReplicaManager.scala | 6 +- .../main/scala/kafka/server/RequestPurgatory.scala | 275 +++++++++----------- .../server/DelayedOperationPurgatoryTest.scala | 131 ---------- .../unit/kafka/server/RequestPurgatoryTest.scala | 135 ++++++++++ 10 files changed, 396 insertions(+), 716 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala delete mode 100644 core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala create mode 100644 core/src/main/scala/kafka/server/FetchRequestPurgatory.scala delete mode 100644 core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala create mode 100644 core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala delete mode 100644 core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala diff --git a/core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala b/core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala deleted file mode 100644 index 3998b6d..0000000 --- a/core/src/main/scala/kafka/server/DelayedOperationPurgatory.scala +++ /dev/null @@ -1,285 +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 kafka.network._ -import kafka.utils._ -import kafka.metrics.KafkaMetricsGroup - -import java.util -import java.util.concurrent._ -import java.util.concurrent.atomic._ -import scala.collection._ - -import com.yammer.metrics.core.Gauge - - -/** - * An operation whose processing needs to be delayed for at most the given delayMs; - * upon complete, the given callback function will be triggered. For example a delayed - * message append operation could be waiting for specified number of acks; or a delayed - * message fetch operation could be waiting for a given number of bytes to accumulate. - */ -abstract class DelayedOperation(delayMs: Long, onComplete: Boolean => Unit) extends DelayedItem(delayMs) { - val completed = new AtomicBoolean(false) - - /* - * Check if the delayed operation is already completed - * - * Note that concurrent threads can check if an operation can be completed or not, - * but only the first thread will succeed in completing the operation - */ - def tryComplete(): Boolean = completed.compareAndSet(false, true) - - /* - * When delayMs has elapsed, expire the delayed operation - */ - def onExpired() = onComplete(false) -} - -/** - * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. - * - */ -abstract class DelayedOperationPurgatory[T <: DelayedOperation](brokerId: Int = 0, purgeInterval: Int = 1000) - extends Logging with KafkaMetricsGroup { - - /* a list of requests watching each key */ - private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) - - /* the number of requests being watched, duplicates added on different watchers are also counted */ - private val watched = new AtomicInteger(0) - - /* background thread expiring requests that have been waiting too long */ - private val expirationReaper = new ExpiredOperationReaper - - newGauge( - "PurgatorySize", - new Gauge[Int] { - def value = size - } - ) - - newGauge( - "NumDelayedOperations", - new Gauge[Int] { - def value = expirationReaper.numOperations - } - ) - - expirationThread.start() - - /** - * Check if the operation can be completed, if not watch it based on the given watch keys - * - * Note that a delayed operation can be watched on multiple keys, and hence due to concurrency may be - * found completed when trying to watch it on some later keys. In this case the operation is still - * treated as completed and hence no longer watched although it is still in the watch lists of - * the earlier keys. Those already watched elements will be later purged by the expire reaper. - * - * @param operation the delayed operation to be checked - * @param watchKeys keys for bookkeeping the operation - * @return true iff the delayed operations can be completed - */ - def tryCompleteElseWatch(operation: DelayedOperation, watchKeys: Seq[Any]): Boolean = { - for(key <- watchKeys) { - val watchers = watchersFor(key) - // if the operation is found completed, stop adding it to any further - // lists and return true immediately - if(!watchers.checkAndMaybeAdd(operation)) { - return true - } - } - - // if it is indeed watched, add to the expire queue also - watched.getAndIncrement() - expirationReaper.enqueue(operation) - - false - - } - - /** - * Return a list of completed operations with the given watch key. - */ - def getCompleted(key: Any): Seq[T] = { - val watchers = watchersForKey.get(key) - if(watchers == null) - Seq.empty - else - watchers.collectCompletedOperations() - } - - /* - * Return the watch list of the given key - */ - private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) - - /* - * Return the size of the purgatory, which is size of watch lists plus the size of the expire reaper. - * Since an operation may still be in the watch lists even when it has been completed, this number - * may be larger than the number of real operations watched - */ - protected def size() = watchersForKey.values.map(_.numRequests).sum + expirationReaper.numOperations - - /** - * Shutdown the expire reaper thread - */ - def shutdown() { - expirationReaper.shutdown() - } - - /** - * A linked list of watched delayed operations based on some key - */ - private class Watchers { - private val requests = new util.ArrayList[T] - - // potentially add the element to watch if it is not satisfied yet - def checkAndMaybeAdd(t: T): Boolean = { - synchronized { - // if it is already satisfied, return false - if (t.completed.get()) - return false - // if the operation can be completed, return false; otherwise add to watch list - if(t.tryComplete()) { - return false - } else { - requests.add(t) - return true - } - } - } - - // traverse the list and purge satisfied elements - def purgeSatisfied(): Int = { - synchronized { - val iter = requests.iterator() - var purged = 0 - while (iter.hasNext) { - val curr = iter.next - if(curr.completed.get()) { - iter.remove() - purged += 1 - } - } - purged - } - } - - // traverse the list and try to satisfy watched elements - def collectCompletedOperations(): Seq[T] = { - val response = new mutable.ArrayBuffer[T] - synchronized { - val iter = requests.iterator() - while(iter.hasNext) { - val curr = iter.next - if (curr.completed.get()) { - // another thread has completed this request, just remove it - iter.remove() - } else { - val completed = curr.tryComplete() - if(completed) { - iter.remove() - watched.getAndDecrement() - response += curr - expirationReaper.satisfyRequest() - } - } - } - } - response - } - } - - /** - * A background reaper to expire delayed operations that have timed out - */ - private class ExpiredOperationReaper extends ShutdownableThread( - "ExpirationReaper-%d".format(brokerId), - false) { - - /* The queue storing all delayed operations */ - private val delayed = new DelayQueue[T] - - /* - * Return the number of delayed operations kept by the reaper - */ - def numOperations = delayed.size() - - /* - * Add a operation to be expired - */ - def enqueue(t: T) { - delayed.add(t) - unsatisfied.incrementAndGet() - } - - /** - * Get the next expired event - */ - private def pollExpired(): T = { - while (true) { - val curr = delayed.poll(200L, TimeUnit.MILLISECONDS) - if (curr == null) - return null.asInstanceOf[T] - // try set the operation failed (and hence completed), if succeed return it; - // otherwise try to get the next expired operation since this one has been completed by others - if (curr.completed.compareAndSet(false, true)) { - return curr - } - } - throw new RuntimeException("This should not happen") - } - - /** - * Delete all satisfied events from the delay queue and the watcher lists - */ - private def purgeSatisfied(): Int = { - var purged = 0 - - // purge the delayed queue - val iter = delayed.iterator() - while (iter.hasNext) { - val curr = iter.next() - if (curr.completed.get()) { - iter.remove() - purged += 1 - } - } - - purged - } - - - override def doWork() { - val curr = pollExpired() - if (curr != null) { - curr.onExpired() - } - if (size >= purgeInterval) { // see if we need to force a full purge - debug("Beginning purgatory purge") - val purged = purgeSatisfied() - debug("Purged %d operations from delay queue.".format(purged)) - val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum - debug("Purged %d operations from watch lists.".format(numPurgedFromWatchers)) - } - } - } - -} diff --git a/core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala b/core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala deleted file mode 100644 index 71c5920..0000000 --- a/core/src/main/scala/kafka/server/FetchDelayedOperationPurgatory.scala +++ /dev/null @@ -1,69 +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 kafka.metrics.KafkaMetricsGroup -import kafka.network.RequestChannel -import kafka.api.FetchResponseSend - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed fetch requests - */ -class FetchDelayedOperationPurgatory(replicaManager: ReplicaManager, requestChannel: RequestChannel) - extends DelayedOperationPurgatory[DelayedFetch](replicaManager.config.brokerId, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) { - this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup { - private val metricPrefix = if (forFollower) "Follower" else "Consumer" - - val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) - private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) - - private def recordDelayedFetchExpired(forFollower: Boolean) { - val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics - else aggregateNonFollowerFetchRequestMetrics - - metrics.expiredRequestMeter.mark() - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedFetch: DelayedFetch): Boolean = delayedFetch.isSatisfied(replicaManager) - - /** - * When a delayed fetch request expires just answer it with whatever data is present - */ - def expire(delayedFetch: DelayedFetch) { - debug("Expiring fetch request %s.".format(delayedFetch.fetch)) - val fromFollower = delayedFetch.fetch.isFromFollower - recordDelayedFetchExpired(fromFollower) - respond(delayedFetch) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedFetch: DelayedFetch) { - val response = delayedFetch.respond(replicaManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedFetch.request, new FetchResponseSend(response))) - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala b/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala new file mode 100644 index 0000000..ed13188 --- /dev/null +++ b/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.metrics.KafkaMetricsGroup +import kafka.network.RequestChannel +import kafka.api.FetchResponseSend + +import java.util.concurrent.TimeUnit + +/** + * The purgatory holding delayed fetch requests + */ +class FetchRequestPurgatory(replicaManager: ReplicaManager, requestChannel: RequestChannel) + extends RequestPurgatory[DelayedFetch](replicaManager.config.brokerId, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) { + this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId) + + private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup { + private val metricPrefix = if (forFollower) "Follower" else "Consumer" + + val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) + } + + private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) + private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) + + private def recordDelayedFetchExpired(forFollower: Boolean) { + val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics + else aggregateNonFollowerFetchRequestMetrics + + metrics.expiredRequestMeter.mark() + } + + /** + * Check if a specified delayed fetch request is satisfied + */ + def checkSatisfied(delayedFetch: DelayedFetch): Boolean = delayedFetch.isSatisfied(replicaManager) + + /** + * When a delayed fetch request expires just answer it with whatever data is present + */ + def expire(delayedFetch: DelayedFetch) { + debug("Expiring fetch request %s.".format(delayedFetch.fetch)) + val fromFollower = delayedFetch.fetch.isFromFollower + recordDelayedFetchExpired(fromFollower) + respond(delayedFetch) + } + + // TODO: purgatory should not be responsible for sending back the responses + def respond(delayedFetch: DelayedFetch) { + val response = delayedFetch.respond(replicaManager) + requestChannel.sendResponse(new RequestChannel.Response(delayedFetch.request, new FetchResponseSend(response))) + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5bee8f8..85498b4 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -42,8 +42,8 @@ class KafkaApis(val requestChannel: RequestChannel, val config: KafkaConfig, val controller: KafkaController) extends Logging { - val producerRequestPurgatory = new ProducerDelayedOperationPurgatory(replicaManager, offsetManager, requestChannel) - val fetchRequestPurgatory = new FetchDelayedOperationPurgatory(replicaManager, requestChannel) + val producerRequestPurgatory = new ProducerRequestPurgatory(replicaManager, offsetManager, requestChannel) + val fetchRequestPurgatory = new FetchRequestPurgatory(replicaManager, requestChannel) // TODO: the following line will be removed in 0.9 replicaManager.initWithRequestPurgatory(producerRequestPurgatory, fetchRequestPurgatory) var metadataCache = new MetadataCache diff --git a/core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala b/core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala deleted file mode 100644 index 4b950e1..0000000 --- a/core/src/main/scala/kafka/server/ProducerDelayedOperationPurgatory.scala +++ /dev/null @@ -1,69 +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 kafka.metrics.KafkaMetricsGroup -import kafka.utils.Pool -import kafka.network.{BoundedByteBufferSend, RequestChannel} - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed producer requests - */ -class ProducerDelayedOperationPurgatory(replicaManager: ReplicaManager, offsetManager: OffsetManager, requestChannel: RequestChannel) - extends DelayedOperationPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) { - this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup { - val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val producerRequestMetricsForKey = { - val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") - new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory)) - } - - private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics - - private def recordDelayedProducerKeyExpired(key: DelayedRequestKey) { - val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) - List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedProduce: DelayedProduce) = delayedProduce.isSatisfied(replicaManager) - - /** - * When a delayed produce request expires answer it with possible time out error codes - */ - def expire(delayedProduce: DelayedProduce) { - debug("Expiring produce request %s.".format(delayedProduce.produce)) - for ((topicPartition, responseStatus) <- delayedProduce.partitionStatus if responseStatus.acksPending) - recordDelayedProducerKeyExpired(new TopicPartitionRequestKey(topicPartition)) - respond(delayedProduce) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedProduce: DelayedProduce) { - val response = delayedProduce.respond(offsetManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedProduce.request, new BoundedByteBufferSend(response))) - } -} diff --git a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala b/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala new file mode 100644 index 0000000..d4a7d4a --- /dev/null +++ b/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.Pool +import kafka.network.{BoundedByteBufferSend, RequestChannel} + +import java.util.concurrent.TimeUnit + +/** + * The purgatory holding delayed producer requests + */ +class ProducerRequestPurgatory(replicaManager: ReplicaManager, offsetManager: OffsetManager, requestChannel: RequestChannel) + extends RequestPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) { + this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId) + + private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup { + val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) + } + + private val producerRequestMetricsForKey = { + val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") + new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory)) + } + + private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics + + private def recordDelayedProducerKeyExpired(key: DelayedRequestKey) { + val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) + List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) + } + + /** + * Check if a specified delayed fetch request is satisfied + */ + def checkSatisfied(delayedProduce: DelayedProduce) = delayedProduce.isSatisfied(replicaManager) + + /** + * When a delayed produce request expires answer it with possible time out error codes + */ + def expire(delayedProduce: DelayedProduce) { + debug("Expiring produce request %s.".format(delayedProduce.produce)) + for ((topicPartition, responseStatus) <- delayedProduce.partitionStatus if responseStatus.acksPending) + recordDelayedProducerKeyExpired(new TopicPartitionRequestKey(topicPartition)) + respond(delayedProduce) + } + + // TODO: purgatory should not be responsible for sending back the responses + def respond(delayedProduce: DelayedProduce) { + val response = delayedProduce.respond(offsetManager) + requestChannel.sendResponse(new RequestChannel.Response(delayedProduce.request, new BoundedByteBufferSend(response))) + } +} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 9eb8d5d..4abf034 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -64,8 +64,8 @@ class ReplicaManager(config: KafkaConfig, this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger - var producerRequestPurgatory: ProducerDelayedOperationPurgatory = null - var fetchRequestPurgatory: FetchDelayedOperationPurgatory = null + var producerRequestPurgatory: ProducerRequestPurgatory = null + var fetchRequestPurgatory: FetchRequestPurgatory = null newGauge( "LeaderCount", @@ -105,7 +105,7 @@ class ReplicaManager(config: KafkaConfig, * TODO: will be removed in 0.9 where we refactor server structure */ - def initWithRequestPurgatory(producerRequestPurgatory: ProducerDelayedOperationPurgatory, fetchRequestPurgatory: FetchDelayedOperationPurgatory) { + def initWithRequestPurgatory(producerRequestPurgatory: ProducerRequestPurgatory, fetchRequestPurgatory: FetchRequestPurgatory) { this.producerRequestPurgatory = producerRequestPurgatory this.fetchRequestPurgatory = fetchRequestPurgatory } diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 9d76234..c55aac1 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -17,7 +17,6 @@ package kafka.server -import kafka.network._ import kafka.utils._ import kafka.metrics.KafkaMetricsGroup @@ -30,39 +29,30 @@ import com.yammer.metrics.core.Gauge /** - * A request whose processing needs to be delayed for at most the given delayMs - * The associated keys are used for bookeeping, and represent the "trigger" that causes this request to check if it is satisfied, - * for example a key could be a (topic, partition) pair. + * An operation whose processing needs to be delayed for at most the given delayMs; + * upon complete, the given callback function will be triggered. For example a delayed + * message append operation could be waiting for specified number of acks; or a delayed + * message fetch operation could be waiting for a given number of bytes to accumulate. */ -class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, delayMs: Long) extends DelayedItem[RequestChannel.Request](request, delayMs) { - val satisfied = new AtomicBoolean(false) +abstract class DelayedRequest(delayMs: Long, onComplete: Boolean => Unit) extends DelayedItem(delayMs) { + val completed = new AtomicBoolean(false) + + /* + * Check if the delayed operation is already completed + * + * Note that concurrent threads can check if an operation can be completed or not, + * but only the first thread will succeed in completing the operation + */ + def tryComplete(): Boolean = completed.compareAndSet(false, true) + + /* + * When delayMs has elapsed, expire the delayed operation + */ + def onExpired() = onComplete(false) } /** - * A helper class for dealing with asynchronous requests with a timeout. A DelayedRequest has a request to delay - * and also a list of keys that can trigger the action. Implementations can add customized logic to control what it means for a given - * request to be satisfied. For example it could be that we are waiting for user-specified number of acks on a given (topic, partition) - * to be able to respond to a request or it could be that we are waiting for a given number of bytes to accumulate on a given request - * to be able to respond to that request (in the simple case we might wait for at least one byte to avoid busy waiting). - * - * For us the key is generally a (topic, partition) pair. - * By calling - * val isSatisfiedByMe = checkAndMaybeWatch(delayedRequest) - * we will check if a request is satisfied already, and if not add the request for watch on all its keys. - * - * It is up to the user to then call - * val satisfied = update(key, request) - * when a request relevant to the given key occurs. This triggers bookeeping logic and returns back any requests satisfied by this - * new request. - * - * An implementation provides extends two helper functions - * def checkSatisfied(request: R, delayed: T): Boolean - * this function returns true if the given request (in combination with whatever previous requests have happened) satisfies the delayed - * request delayed. This method will likely also need to do whatever bookkeeping is necessary. - * - * The second function is - * def expire(delayed: T) - * this function handles delayed requests that have hit their time limit without being satisfied. + * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. * */ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: Int = 1000) @@ -71,119 +61,109 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt /* a list of requests watching each key */ private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) + /* the number of requests being watched, duplicates added on different watchers are also counted */ + private val watched = new AtomicInteger(0) + /* background thread expiring requests that have been waiting too long */ - private val expiredRequestReaper = new ExpiredRequestReaper - private val expirationThread = Utils.newThread(name="request-expiration-task", runnable=expiredRequestReaper, daemon=false) + private val expirationReaper = new ExpiredOperationReaper newGauge( "PurgatorySize", new Gauge[Int] { - def value = watched() + def value = size() } ) newGauge( - "NumDelayedRequests", + "NumDelayedOperations", new Gauge[Int] { - def value = delayed() + def value = expirationReaper.numOperations } ) expirationThread.start() /** - * Try to add the request for watch on all keys. Return true iff the request is - * satisfied and the satisfaction is done by the caller. + * Check if the operation can be completed, if not watch it based on the given watch keys + * + * Note that a delayed operation can be watched on multiple keys, and hence due to concurrency may be + * found completed when trying to watch it on some later keys. In this case the operation is still + * treated as completed and hence no longer watched although it is still in the watch lists of + * the earlier keys. Those already watched elements will be later purged by the expire reaper. * - * Requests can be watched on only a few of the keys if it is found satisfied when - * trying to add it to each one of the keys. In this case the request is still treated as satisfied - * and hence no longer watched. Those already added elements will be later purged by the expire reaper. + * @param operation the delayed operation to be checked + * @param watchKeys keys for bookkeeping the operation + * @return true iff the delayed operations can be completed */ - def checkAndMaybeWatch(delayedRequest: T): Boolean = { - for(key <- delayedRequest.keys) { - val lst = watchersFor(key) - if(!lst.checkAndMaybeAdd(delayedRequest)) { - if(delayedRequest.satisfied.compareAndSet(false, true)) - return true - else - return false + def tryCompleteElseWatch(operation: DelayedRequest, watchKeys: Seq[Any]): Boolean = { + for(key <- watchKeys) { + val watchers = watchersFor(key) + // if the operation is found completed, stop adding it to any further + // lists and return true immediately + if(!watchers.checkAndMaybeAdd(operation)) { + return true } } // if it is indeed watched, add to the expire queue also - expiredRequestReaper.enqueue(delayedRequest) + watched.getAndIncrement() + expirationReaper.enqueue(operation) false + } /** - * Update any watchers and return a list of newly satisfied requests. + * Return a list of completed operations with the given watch key. + * + * @return the list of completed operations */ - def update(key: Any): Seq[T] = { - val w = watchersForKey.get(key) - if(w == null) + def getCompleted(key: Any): Seq[T] = { + val watchers = watchersForKey.get(key) + if(watchers == null) Seq.empty else - w.collectSatisfiedRequests() + watchers.collectCompletedOperations() } /* - * Return the size of the watched lists in the purgatory, which is the size of watch lists. - * Since an operation may still be in the watch lists even when it has been completed, - * this number may be larger than the number of real operations watched + * Return the watch list of the given key */ - def watched() = watchersForKey.values.map(_.watched).sum - - /* - * Return the number of requests in the expiry reaper's queue - */ - def delayed() = expiredRequestReaper.delayed() + private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) /* - * Return the watch list for the given watch key + * Return the size of the purgatory, which is size of watch lists plus the size of the expire reaper. + * Since an operation may still be in the watch lists even when it has been completed, this number + * may be larger than the number of real operations watched */ - private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) + protected def size() = watchersForKey.values.map(_.numRequests).sum + expirationReaper.numOperations /** - * Check if this delayed request is already satisfied - */ - protected def checkSatisfied(request: T): Boolean - - /** - * Handle an expired delayed request - */ - protected def expire(delayed: T) - - /** * Shutdown the expire reaper thread */ def shutdown() { - expiredRequestReaper.shutdown() + expirationReaper.shutdown() } /** - * A linked list of DelayedRequests watching some key with some associated - * bookkeeping logic. + * A linked list of watched delayed operations based on some key */ private class Watchers { private val requests = new util.LinkedList[T] - // return the size of the watch list - def watched() = requests.size() - // potentially add the element to watch if it is not satisfied yet def checkAndMaybeAdd(t: T): Boolean = { synchronized { - // if it is already satisfied, do not add to the watch list - if (t.satisfied.get) + // if it is already satisfied, return false + if (t.completed.get()) return false - // synchronize on the delayed request to avoid any race condition - // with expire and update threads on client-side. - if(t synchronized checkSatisfied(t)) { + // if the operation can be completed, return false; otherwise add to watch list + if(t.tryComplete()) { return false + } else { + requests.add(t) + return true } - requests.add(t) - return true } } @@ -192,9 +172,9 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt synchronized { val iter = requests.iterator() var purged = 0 - while(iter.hasNext) { + while (iter.hasNext) { val curr = iter.next - if(curr.satisfied.get()) { + if(curr.completed.get()) { iter.remove() purged += 1 } @@ -204,25 +184,22 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt } // traverse the list and try to satisfy watched elements - def collectSatisfiedRequests(): Seq[T] = { + def collectCompletedOperations(): Seq[T] = { val response = new mutable.ArrayBuffer[T] synchronized { val iter = requests.iterator() while(iter.hasNext) { val curr = iter.next - if(curr.satisfied.get) { - // another thread has satisfied this request, remove it + if (curr.completed.get()) { + // another thread has completed this request, just remove it iter.remove() } else { - // synchronize on curr to avoid any race condition with expire - // on client-side. - val satisfied = curr synchronized checkSatisfied(curr) - if(satisfied) { + val completed = curr.tryComplete() + if(completed) { iter.remove() - val updated = curr.satisfied.compareAndSet(false, true) - if(updated == true) { - response += curr - } + watched.getAndDecrement() + response += curr + expirationReaper.satisfyRequest() } } } @@ -232,70 +209,39 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt } /** - * Runnable to expire requests that have sat unfullfilled past their deadline + * A background reaper to expire delayed operations that have timed out */ - private class ExpiredRequestReaper extends Runnable with Logging { - this.logIdent = "ExpiredRequestReaper-%d ".format(brokerId) - private val running = new AtomicBoolean(true) - private val shutdownLatch = new CountDownLatch(1) - - private val delayedQueue = new DelayQueue[T] - - def delayed() = delayedQueue.size() - - /** Main loop for the expiry thread */ - def run() { - while(running.get) { - try { - val curr = pollExpired() - if (curr != null) { - curr synchronized { - expire(curr) - } - } - // see if we need to purge the watch lists - if (RequestPurgatory.this.watched() >= purgeInterval) { - debug("Begin purging watch lists") - val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum - debug("Purged %d elements from watch lists.".format(numPurgedFromWatchers)) - } - // see if we need to purge the delayed request queue - if (delayed() >= purgeInterval) { - debug("Begin purging delayed queue") - val purged = purgeSatisfied() - debug("Purged %d requests from delayed queue.".format(purged)) - } - } catch { - case e: Exception => - error("Error in long poll expiry thread: ", e) - } - } - shutdownLatch.countDown() - } + private class ExpiredOperationReaper extends ShutdownableThread( + "ExpirationReaper-%d".format(brokerId), + false) { - /** Add a request to be expired */ - def enqueue(t: T) { - delayedQueue.add(t) - } + /* The queue storing all delayed operations */ + private val delayed = new DelayQueue[T] - /** Shutdown the expiry thread*/ - def shutdown() { - debug("Shutting down.") - running.set(false) - shutdownLatch.await() - debug("Shut down complete.") + /* + * Return the number of delayed operations kept by the reaper + */ + def numOperations = delayed.size() + + /* + * Add a operation to be expired + */ + def enqueue(t: T) { + delayed.add(t) + unsatisfied.incrementAndGet() } /** * Get the next expired event */ private def pollExpired(): T = { - while(true) { - val curr = delayedQueue.poll(200L, TimeUnit.MILLISECONDS) + while (true) { + val curr = delayed.poll(200L, TimeUnit.MILLISECONDS) if (curr == null) return null.asInstanceOf[T] - val updated = curr.satisfied.compareAndSet(false, true) - if(updated) { + // try set the operation failed (and hence completed), if succeed return it; + // otherwise try to get the next expired operation since this one has been completed by others + if (curr.completed.compareAndSet(false, true)) { return curr } } @@ -309,10 +255,10 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt var purged = 0 // purge the delayed queue - val iter = delayedQueue.iterator() - while(iter.hasNext) { + val iter = delayed.iterator() + while (iter.hasNext) { val curr = iter.next() - if(curr.satisfied.get) { + if (curr.completed.get()) { iter.remove() purged += 1 } @@ -320,6 +266,21 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt purged } + + + override def doWork() { + val curr = pollExpired() + if (curr != null) { + curr.onExpired() + } + if (size() >= purgeInterval) { // see if we need to force a full purge + debug("Beginning purgatory purge") + val purged = purgeSatisfied() + debug("Purged %d operations from delay queue.".format(purged)) + val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum + debug("Purged %d operations from watch lists.".format(numPurgedFromWatchers)) + } + } } } diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala deleted file mode 100644 index ff332d3..0000000 --- a/core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala +++ /dev/null @@ -1,131 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import scala.collection._ -import org.junit.Test -import junit.framework.Assert._ -import kafka.message._ -import kafka.api._ -import kafka.utils.TestUtils -import org.scalatest.junit.JUnit3Suite - - -class DelayedOperationPurgatoryTest extends JUnit3Suite { - - val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes))) - val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes))) - var purgatory: MockDelayedOperationPurgatory = null - - override def setUp() { - super.setUp() -<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala - purgatory = new MockRequestPurgatory(5) -======= - purgatory = new MockDelayedOperationPurgatory() ->>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala - } - - override def tearDown() { - purgatory.shutdown() - super.tearDown() - } - - @Test - def testRequestSatisfaction() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r2 = new DelayedRequest(Array("test2"), null, 100000L) - assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.update("test1").size) - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test1").size) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test2").size) - purgatory.satisfied += r1 - assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1")) - assertEquals("Nothing satisfied", 0, purgatory.update("test1").size) - purgatory.satisfied += r2 - assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2")) - assertEquals("Nothing satisfied", 0, purgatory.update("test2").size) - } - - @Test - def testRequestExpiry() { - val expiration = 20L - val r1 = new DelayedRequest(Array("test1"), null, expiration) - val r2 = new DelayedRequest(Array("test1"), null, 200000L) - val start = System.currentTimeMillis - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - purgatory.awaitExpiration(r1) - val elapsed = System.currentTimeMillis - start - assertTrue("r1 expired", purgatory.expired.contains(r1)) - assertTrue("r2 hasn't expired", !purgatory.expired.contains(r2)) - assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) - } - - @Test - def testRequestPurge() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r12 = new DelayedRequest(Array("test1", "test2"), null, 100000L) - val r23 = new DelayedRequest(Array("test2", "test3"), null, 100000L) - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r12) - purgatory.checkAndMaybeWatch(r23) - - assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) - assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) - - // satisfy one of the requests, it should then be purged from the watch list with purge interval 5 - r12.satisfied.set(true) - TestUtils.waitUntilTrue(() => purgatory.watched() == 3, - "Purgatory should have 3 watched elements instead of " + + purgatory.watched(), 1000L) - TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, - "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) - - // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r1) - - TestUtils.waitUntilTrue(() => purgatory.watched() == 5, - "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) - TestUtils.waitUntilTrue(() => purgatory.delayed() == 4, - "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) - } - -<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala - class MockRequestPurgatory(purge: Int) extends RequestPurgatory[DelayedRequest](purgeInterval = purge) { -======= - class MockDelayedOperationPurgatory extends DelayedOperationPurgatory[DelayedRequest] { ->>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala - val satisfied = mutable.Set[DelayedRequest]() - val expired = mutable.Set[DelayedRequest]() - def awaitExpiration(delayed: DelayedRequest) = { - delayed synchronized { - delayed.wait() - } - } - def checkSatisfied(delayed: DelayedRequest): Boolean = satisfied.contains(delayed) - def expire(delayed: DelayedRequest) { - expired += delayed - delayed synchronized { - delayed.notify() - } - } - } - -} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala new file mode 100644 index 0000000..bdeed80 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala @@ -0,0 +1,135 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 scala.collection._ +import org.junit.Test +import junit.framework.Assert._ +import kafka.message._ +import kafka.api._ +import kafka.utils.TestUtils +import org.scalatest.junit.JUnit3Suite + + +class RequestPurgatoryTest extends JUnit3Suite { + + val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes))) + val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes))) + var purgatory: MockDelayedOperationPurgatory = null + + override def setUp() { + super.setUp() +<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala + purgatory = new MockRequestPurgatory(5) +======= + purgatory = new MockDelayedOperationPurgatory() +>>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala + } + + override def tearDown() { + purgatory.shutdown() + super.tearDown() + } + + @Test + def testRequestSatisfaction() { + val r1 = new DelayedRequest(Array("test1"), null, 100000L) + val r2 = new DelayedRequest(Array("test2"), null, 100000L) + assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.update("test1").size) + assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) + assertEquals("Still nothing satisfied", 0, purgatory.update("test1").size) + assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) + assertEquals("Still nothing satisfied", 0, purgatory.update("test2").size) + purgatory.satisfied += r1 + assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1")) + assertEquals("Nothing satisfied", 0, purgatory.update("test1").size) + purgatory.satisfied += r2 + assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2")) + assertEquals("Nothing satisfied", 0, purgatory.update("test2").size) + } + + @Test + def testRequestExpiry() { + val expiration = 20L + val r1 = new DelayedRequest(Array("test1"), null, expiration) + val r2 = new DelayedRequest(Array("test1"), null, 200000L) + val start = System.currentTimeMillis + assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) + assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) + purgatory.awaitExpiration(r1) + val elapsed = System.currentTimeMillis - start + assertTrue("r1 expired", purgatory.expired.contains(r1)) + assertTrue("r2 hasn't expired", !purgatory.expired.contains(r2)) + assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) + } + + @Test + def testRequestPurge() { + val r1 = new DelayedRequest(Array("test1"), null, 100000L) + val r12 = new DelayedRequest(Array("test1", "test2"), null, 100000L) + val r23 = new DelayedRequest(Array("test2", "test3"), null, 100000L) + purgatory.checkAndMaybeWatch(r1) + purgatory.checkAndMaybeWatch(r12) + purgatory.checkAndMaybeWatch(r23) + + assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) + assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) + + // satisfy one of the requests, it should then be purged from the watch list with purge interval 5 + r12.satisfied.set(true) + TestUtils.waitUntilTrue(() => purgatory.watched() == 3, + "Purgatory should have 3 watched elements instead of " + + purgatory.watched(), 1000L) + TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, + "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) + + // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 + purgatory.checkAndMaybeWatch(r1) + purgatory.checkAndMaybeWatch(r1) + + TestUtils.waitUntilTrue(() => purgatory.watched() == 5, + "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) + TestUtils.waitUntilTrue(() => purgatory.delayed() == 4, + "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) + } + +<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala +<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala + class MockRequestPurgatory(purge: Int) extends RequestPurgatory[DelayedRequest](purgeInterval = purge) { +======= + class MockDelayedOperationPurgatory extends DelayedOperationPurgatory[DelayedRequest] { +>>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala +======= + class MockRequestPurgatory extends RequestPurgatory[DelayedRequest] { +>>>>>>> change name back:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala + val satisfied = mutable.Set[DelayedRequest]() + val expired = mutable.Set[DelayedRequest]() + def awaitExpiration(delayed: DelayedRequest) = { + delayed synchronized { + delayed.wait() + } + } + def checkSatisfied(delayed: DelayedRequest): Boolean = satisfied.contains(delayed) + def expire(delayed: DelayedRequest) { + expired += delayed + delayed synchronized { + delayed.notify() + } + } + } + +} \ No newline at end of file -- 1.7.12.4 From fb1d443ec3e0ba7195f4d65b6e46cc7e50343759 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 13 Aug 2014 09:13:06 -0700 Subject: [PATCH 03/33] dummy --- .../src/main/scala/kafka/server/DelayedFetch.scala | 39 ++++++++++++---------- core/src/main/scala/kafka/server/KafkaApis.scala | 16 +++++++-- 2 files changed, 36 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index e0f14e2..d367786 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -37,34 +37,39 @@ import scala.collection.Seq * - should return whatever data is available. */ -class DelayedFetch(override val keys: Seq[TopicPartitionRequestKey], - override val request: RequestChannel.Request, - override val delayMs: Long, - val fetch: FetchRequest, - private val partitionFetchOffsets: Map[TopicAndPartition, LogOffsetMetadata]) - extends DelayedRequest(keys, request, delayMs) { +case class FetchInfo(fetchMinBytes: Int, + fetchOnlyCommitted: Boolean, + fetchStartOffsets: Map[TopicAndPartition, LogOffsetMetadata]) { - def isSatisfied(replicaManager: ReplicaManager) : Boolean = { + override def toString = "FetchInfo [minBytes: " + fetchMinBytes + "] : " + + "[committedOnly: " + fetchOnlyCommitted + "] : " + "[startOffsets: " + fetchStartOffsets + "]" +} + + +class DelayedFetch(delayMs: Long, onComplete: Boolean => Unit, fetchInfo: FetchInfo, replicaManager: ReplicaManager) + extends DelayedRequest(delayMs, onComplete) { + + override def tryComplete() : Boolean = { var accumulatedSize = 0 - val fromFollower = fetch.isFromFollower - partitionFetchOffsets.foreach { + fetchInfo.fetchStartOffsets.foreach { case (topicAndPartition, fetchOffset) => try { if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) { val replica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) val endOffset = - if (fromFollower) - replica.logEndOffset - else + if (fetchInfo.fetchOnlyCommitted) replica.highWatermark + else + replica.logEndOffset if (endOffset.offsetOnOlderSegment(fetchOffset)) { // Case C, this can happen when the new follower replica fetching on a truncated leader - debug("Satisfying fetch request %s since it is fetching later segments of partition %s.".format(fetch, topicAndPartition)) + debug("Satisfying %s since it is fetching later segments of partition %s.".format(fetchInfo, topicAndPartition)) return true } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { // Case C, this can happen when the folloer replica is lagging too much - debug("Satisfying fetch request %s immediately since it is fetching older segments.".format(fetch)) + debug("Satisfying %s immediately since it is fetching older segments.".format(fetchInfo)) return true } else if (fetchOffset.precedes(endOffset)) { accumulatedSize += endOffset.positionDiff(fetchOffset) @@ -72,16 +77,16 @@ class DelayedFetch(override val keys: Seq[TopicPartitionRequestKey], } } catch { case utpe: UnknownTopicOrPartitionException => // Case A - debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetch)) + debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchInfo)) return true case nle: NotLeaderForPartitionException => // Case B - debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetch)) + debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchInfo)) return true } } // Case D - accumulatedSize >= fetch.minBytes + accumulatedSize >= fetchInfo.fetchMinBytes } def respond(replicaManager: ReplicaManager): FetchResponse = { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 85498b4..80ee95c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -331,8 +331,20 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.clientId)) // create a list of (topic, partition) pairs to use as keys for this delayed request val delayedFetchKeys = fetchRequest.requestInfo.keys.toSeq.map(new TopicPartitionRequestKey(_)) - val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest.maxWait, fetchRequest, - dataRead.mapValues(_.offset)) + + //val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest.maxWait, fetchRequest, + // dataRead.mapValues(_.offset)) + + def callback(succeeded: Boolean) { + if (succeeded) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(FetchResponse(fetch.correlationId, replicaManager.readMessageSets(fetchRequest).mapValues(_.data))))) + } + + val delayedFetch = new DelayedFetch( + fetchRequest.maxWait, + callback, + dataRead.mapValues(_.offset), + replicaManager) + // add the fetch request for watch if it's not satisfied, otherwise send the response back val satisfiedByMe = fetchRequestPurgatory.checkAndMaybeWatch(delayedFetch) -- 1.7.12.4 From a1390fa6a2e840dcbd4a26288ea9b4f6ee184125 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 13 Aug 2014 15:48:13 -0700 Subject: [PATCH 04/33] wip version 1 --- .../main/scala/kafka/api/ProducerResponse.scala | 3 +- core/src/main/scala/kafka/log/Log.scala | 26 +- .../src/main/scala/kafka/server/DelayedFetch.scala | 25 +- .../main/scala/kafka/server/DelayedProduce.scala | 108 ++++---- core/src/main/scala/kafka/server/KafkaApis.scala | 234 +++-------------- .../main/scala/kafka/server/ReplicaManager.scala | 291 ++++++++++++++++----- .../main/scala/kafka/server/RequestPurgatory.scala | 24 +- 7 files changed, 375 insertions(+), 336 deletions(-) diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index a286272..5d1fac4 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -43,8 +43,7 @@ object ProducerResponse { case class ProducerResponseStatus(var error: Short, offset: Long) -case class ProducerResponse(correlationId: Int, - status: Map[TopicAndPartition, ProducerResponseStatus]) +case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, ProducerResponseStatus]) extends RequestOrResponse() { /** diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 157d673..b381bb1 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -31,6 +31,21 @@ import scala.collection.JavaConversions import com.yammer.metrics.core.Gauge +/** + * Struct to hold various quantities we compute about each message set before appending to the log + * @param firstOffset The first offset in the message set + * @param lastOffset The last offset in the message set + * @param shallowCount The number of shallow messages + * @param validBytes The number of valid bytes + * @param codec The codec used in the message set + * @param offsetsMonotonic Are the offsets in this message set monotonically increasing + */ +case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) + +object LogAppendInfo { + val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, -1, -1, false) +} + /** * An append-only log for storing messages. @@ -311,17 +326,6 @@ class Log(val dir: File, } /** - * Struct to hold various quantities we compute about each message set before appending to the log - * @param firstOffset The first offset in the message set - * @param lastOffset The last offset in the message set - * @param shallowCount The number of shallow messages - * @param validBytes The number of valid bytes - * @param codec The codec used in the message set - * @param offsetsMonotonic Are the offsets in this message set monotonically increasing - */ - case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) - - /** * Validate the following: *
    *
  1. each message matches its CRC diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index d367786..4b7542f 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -38,6 +38,7 @@ import scala.collection.Seq */ case class FetchInfo(fetchMinBytes: Int, + fetchOnlyLeader: Boolean, fetchOnlyCommitted: Boolean, fetchStartOffsets: Map[TopicAndPartition, LogOffsetMetadata]) { @@ -47,7 +48,10 @@ case class FetchInfo(fetchMinBytes: Int, } -class DelayedFetch(delayMs: Long, onComplete: Boolean => Unit, fetchInfo: FetchInfo, replicaManager: ReplicaManager) +class DelayedFetch(delayMs: Long, + fetchInfo: FetchInfo, + replicaManager: ReplicaManager, + onComplete: Map[TopicAndPartition, PartitionDataAndOffset] => Unit) extends DelayedRequest(delayMs, onComplete) { override def tryComplete() : Boolean = { @@ -66,11 +70,11 @@ class DelayedFetch(delayMs: Long, onComplete: Boolean => Unit, fetchInfo: FetchI if (endOffset.offsetOnOlderSegment(fetchOffset)) { // Case C, this can happen when the new follower replica fetching on a truncated leader debug("Satisfying %s since it is fetching later segments of partition %s.".format(fetchInfo, topicAndPartition)) - return true + return super.tryComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { // Case C, this can happen when the folloer replica is lagging too much debug("Satisfying %s immediately since it is fetching older segments.".format(fetchInfo)) - return true + return super.tryComplete() } else if (fetchOffset.precedes(endOffset)) { accumulatedSize += endOffset.positionDiff(fetchOffset) } @@ -78,15 +82,24 @@ class DelayedFetch(delayMs: Long, onComplete: Boolean => Unit, fetchInfo: FetchI } catch { case utpe: UnknownTopicOrPartitionException => // Case A debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchInfo)) - return true + return super.tryComplete() case nle: NotLeaderForPartitionException => // Case B debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchInfo)) - return true + return super.tryComplete() } } // Case D - accumulatedSize >= fetchInfo.fetchMinBytes + if (accumulatedSize >= fetchInfo.fetchMinBytes) + super.tryComplete() + else + false + } + + override def onExpired() { + // read whatever data is available and return + val readData = replicaManager.readMessageSets(fetch) + onComplete(readData) } def respond(replicaManager: ReplicaManager): FetchResponse = { diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 9481508..99bdf6f 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -35,81 +35,93 @@ import scala.collection.Seq * B.2 - else, at least requiredAcks replicas should be caught up to this request. */ -class DelayedProduce(override val keys: Seq[TopicPartitionRequestKey], - override val request: RequestChannel.Request, - override val delayMs: Long, - val produce: ProducerRequest, - val partitionStatus: Map[TopicAndPartition, DelayedProduceResponseStatus], - val offsetCommitRequestOpt: Option[OffsetCommitRequest] = None) - extends DelayedRequest(keys, request, delayMs) with Logging { +case class ProduceStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { + @volatile var acksPending = false - // first update the acks pending variable according to the error code - partitionStatus foreach { case (topicAndPartition, delayedStatus) => - if (delayedStatus.responseStatus.error == ErrorMapping.NoError) { - // Timeout error state will be cleared when required acks are received - delayedStatus.acksPending = true - delayedStatus.responseStatus.error = ErrorMapping.RequestTimedOutCode - } else { - delayedStatus.acksPending = false - } + override def toString = "acksPending:%b, error: %d, startOffset: %d, requiredOffset: %d" + .format(acksPending, responseStatus.error, responseStatus.offset, requiredOffset) +} - trace("Initial partition status for %s is %s".format(topicAndPartition, delayedStatus)) - } +case class ProduceInfo(produceRequiredAcks: Short, + produceStatus: Map[TopicAndPartition, ProduceStatus]) { - def respond(offsetManager: OffsetManager): RequestOrResponse = { - val responseStatus = partitionStatus.mapValues(status => status.responseStatus) + override def toString = "ProduceInfo [requiredBytes: " + fetchMinBytes + "] : " + + "[partitionStatus: " + produceStatus + "]" +} - val errorCode = responseStatus.find { case (_, status) => - status.error != ErrorMapping.NoError - }.map(_._2.error).getOrElse(ErrorMapping.NoError) +class DelayedProduce(delayMs: Long, + produceInfo: ProduceInfo, + replicaManager: ReplicaManager, + onComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) + extends DelayedRequest(delayMs) with Logging { - if (errorCode == ErrorMapping.NoError) { - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) + // first update the acks pending variable according to the error code + produceInfo.produceStatus foreach { case (topicAndPartition, status) => + if (status.responseStatus.error == ErrorMapping.NoError) { + // Timeout error state will be cleared when required acks are received + status.acksPending = true + status.responseStatus.error = ErrorMapping.RequestTimedOutCode + } else { + status.acksPending = false } - val response = offsetCommitRequestOpt.map(_.responseFor(errorCode, offsetManager.config.maxMetadataSize)) - .getOrElse(ProducerResponse(produce.correlationId, responseStatus)) - - response + trace("Initial partition status for %s is %s".format(topicAndPartition, status)) } - def isSatisfied(replicaManager: ReplicaManager) = { + def tryComplete(): Boolean = { // check for each partition if it still has pending acks - partitionStatus.foreach { case (topicAndPartition, fetchPartitionStatus) => + produceInfo.produceStatus.foreach { case (topicAndPartition, status) => trace("Checking producer request satisfaction for %s, acksPending = %b" - .format(topicAndPartition, fetchPartitionStatus.acksPending)) + .format(topicAndPartition, status.acksPending)) // skip those partitions that have already been satisfied - if (fetchPartitionStatus.acksPending) { + if (status.acksPending) { val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) val (hasEnough, errorCode) = partitionOpt match { case Some(partition) => partition.checkEnoughReplicasReachOffset( - fetchPartitionStatus.requiredOffset, - produce.requiredAcks) + status.requiredOffset, + produceInfo.produceRequiredAcks) case None => (false, ErrorMapping.UnknownTopicOrPartitionCode) } if (errorCode != ErrorMapping.NoError) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.responseStatus.error = errorCode + status.acksPending = false + status.responseStatus.error = errorCode } else if (hasEnough) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.responseStatus.error = ErrorMapping.NoError + status.acksPending = false + status.responseStatus.error = ErrorMapping.NoError } } } // unblocked if there are no partitions with pending acks - val satisfied = ! partitionStatus.exists(p => p._2.acksPending) - satisfied + if (! produceInfo.produceStatus.values.exists(p => p.acksPending)) + super.tryComplete() + else + false } -} -case class DelayedProduceResponseStatus(val requiredOffset: Long, - val responseStatus: ProducerResponseStatus) { - @volatile var acksPending = false + override def onExpired() { + // return the current response status + val responseStatus = produceInfo.produceStatus.mapValues(status => status.responseStatus) + onComplete(responseStatus) + } + + def respond(offsetManager: OffsetManager): RequestOrResponse = { + val responseStatus = partitionStatus.mapValues(status => status.responseStatus) + + val errorCode = responseStatus.find { case (_, status) => + status.error != ErrorMapping.NoError + }.map(_._2.error).getOrElse(ErrorMapping.NoError) - override def toString = - "acksPending:%b, error: %d, startOffset: %d, requiredOffset: %d".format( - acksPending, responseStatus.error, responseStatus.offset, requiredOffset) + if (errorCode == ErrorMapping.NoError) { + offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) + } + + val response = offsetCommitRequestOpt.map(_.responseFor(errorCode, offsetManager.config.maxMetadataSize)) + .getOrElse(ProducerResponse(produce.correlationId, responseStatus)) + + response + } } + diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 80ee95c..64d190c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -151,219 +151,61 @@ class KafkaApis(val requestChannel: RequestChannel, * Handle a produce request or offset commit request (which is really a specialized producer request) */ def handleProducerOrOffsetCommitRequest(request: RequestChannel.Request) { - val (produceRequest, offsetCommitRequestOpt) = - if (request.requestId == RequestKeys.OffsetCommitKey) { - val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - (producerRequestFromOffsetCommit(offsetCommitRequest), Some(offsetCommitRequest)) - } else { - (request.requestObj.asInstanceOf[ProducerRequest], None) - } - - val sTime = SystemTime.milliseconds - val localProduceResults = appendToLocalLog(produceRequest, offsetCommitRequestOpt.nonEmpty) - debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) - - val firstErrorCode = localProduceResults.find(_.errorCode != ErrorMapping.NoError).map(_.errorCode).getOrElse(ErrorMapping.NoError) - - val numPartitionsInError = localProduceResults.count(_.error.isDefined) - if(produceRequest.requiredAcks == 0) { - // no operation needed if producer request.required.acks = 0; however, if there is any exception in handling the request, since - // no response is expected by the producer the handler will send a close connection response to the socket server - // to close the socket so that the producer client will know that some exception has happened and will refresh its metadata - if (numPartitionsInError != 0) { - info(("Send the close connection response due to error handling produce request " + - "[clientId = %s, correlationId = %s, topicAndPartition = %s] with Ack=0") - .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) - requestChannel.closeConnection(request.processor, request) - } else { - - if (firstErrorCode == ErrorMapping.NoError) - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo)) - - if (offsetCommitRequestOpt.isDefined) { - val response = offsetCommitRequestOpt.get.responseFor(firstErrorCode, config.offsetMetadataMaxSize) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) - } else + val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] + + // the callback for sending the response + def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + val numPartitionsInError = responseStatus.values.count(_.error.isDefined) + + if(produceRequest.requiredAcks == 0) { + // no operation needed if producer request.required.acks = 0; however, if there is any exception in handling the request, since + // no response is expected by the producer the handler will send a close connection response to the socket server + // to close the socket so that the producer client will know that some exception has happened and will refresh its metadata + if (numPartitionsInError != 0) { + info(("Send the close connection response due to error handling produce request " + + "[clientId = %s, correlationId = %s, topicAndPartition = %s] with Ack=0") + .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) + requestChannel.closeConnection(request.processor, request) + } else { requestChannel.noOperation(request.processor, request) + } + } else { + val response = ProducerResponse(produceRequest.correlationId, responseStatus) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } - } else if (produceRequest.requiredAcks == 1 || - produceRequest.numPartitions <= 0 || - numPartitionsInError == produceRequest.numPartitions) { - - if (firstErrorCode == ErrorMapping.NoError) { - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) - } - - val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap - val response = offsetCommitRequestOpt.map(_.responseFor(firstErrorCode, config.offsetMetadataMaxSize)) - .getOrElse(ProducerResponse(produceRequest.correlationId, statuses)) - - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) - } else { - // create a list of (topic, partition) pairs to use as keys for this delayed request - val producerRequestKeys = produceRequest.data.keys.map( - topicAndPartition => new TopicPartitionRequestKey(topicAndPartition)).toSeq - val statuses = localProduceResults.map(r => - r.key -> DelayedProduceResponseStatus(r.end + 1, ProducerResponseStatus(r.errorCode, r.start))).toMap - val delayedRequest = new DelayedProduce( - producerRequestKeys, - request, - produceRequest.ackTimeoutMs.toLong, - produceRequest, - statuses, - offsetCommitRequestOpt) - - // add the produce request for watch if it's not satisfied, otherwise send the response back - val satisfiedByMe = producerRequestPurgatory.checkAndMaybeWatch(delayedRequest) - if (satisfiedByMe) - producerRequestPurgatory.respond(delayedRequest) } + // call the replica manager to append messages to the replicas + replicaManager.appendMessages( + produceRequest.ackTimeoutMs.toLong, + produceRequest.requiredAcks, + produceRequest.data, + sendResponseCallback) + // we do not need the data anymore produceRequest.emptyData() } - case class ProduceResult(key: TopicAndPartition, start: Long, end: Long, error: Option[Throwable] = None) { - def this(key: TopicAndPartition, throwable: Throwable) = - this(key, -1L, -1L, Some(throwable)) - - def errorCode = error match { - case None => ErrorMapping.NoError - case Some(error) => ErrorMapping.codeFor(error.getClass.asInstanceOf[Class[Throwable]]) - } - } - - /** - * Helper method for handling a parsed producer request - */ - private def appendToLocalLog(producerRequest: ProducerRequest, isOffsetCommit: Boolean): Iterable[ProduceResult] = { - val partitionAndData: Map[TopicAndPartition, MessageSet] = producerRequest.data - trace("Append [%s] to local log ".format(partitionAndData.toString)) - partitionAndData.map {case (topicAndPartition, messages) => - try { - if (Topic.InternalTopics.contains(topicAndPartition.topic) && - !(isOffsetCommit && topicAndPartition.topic == OffsetManager.OffsetsTopicName)) { - throw new InvalidTopicException("Cannot append to internal topic %s".format(topicAndPartition.topic)) - } - val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) - val info = partitionOpt match { - case Some(partition) => - partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet],producerRequest.requiredAcks) - case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" - .format(topicAndPartition, brokerId)) - } - - val numAppendedMessages = if (info.firstOffset == -1L || info.lastOffset == -1L) 0 else (info.lastOffset - info.firstOffset + 1) - - // update stats for successfully appended bytes and messages as bytesInRate and messageInRate - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) - BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) - - trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) - ProduceResult(topicAndPartition, info.firstOffset, info.lastOffset) - } catch { - // NOTE: Failed produce requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException - // since failed produce requests metric is supposed to indicate failure of a broker in handling a produce request - // for a partition it is the leader for - case e: KafkaStorageException => - fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) - Runtime.getRuntime.halt(1) - null - case ite: InvalidTopicException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, ite.getMessage)) - new ProduceResult(topicAndPartition, ite) - case utpe: UnknownTopicOrPartitionException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, utpe.getMessage)) - new ProduceResult(topicAndPartition, utpe) - case nle: NotLeaderForPartitionException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage)) - new ProduceResult(topicAndPartition, nle) - case nere: NotEnoughReplicasException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nere.getMessage)) - new ProduceResult(topicAndPartition, nere) - case e: Throwable => - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() - error("Error processing ProducerRequest with correlation id %d from client %s on partition %s" - .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e) - new ProduceResult(topicAndPartition, e) - } - } - } - /** * Handle a fetch request */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - val dataRead = replicaManager.readMessageSets(fetchRequest) - - // if the fetch request comes from the follower, - // update its corresponding log end offset - if(fetchRequest.isFromFollower) - recordFollowerLogEndOffsets(fetchRequest.replicaId, dataRead.mapValues(_.offset)) - - // check if this fetch request can be satisfied right away - val bytesReadable = dataRead.values.map(_.data.messages.sizeInBytes).sum - val errorReadingData = dataRead.values.foldLeft(false)((errorIncurred, dataAndOffset) => - errorIncurred || (dataAndOffset.data.error != ErrorMapping.NoError)) - // send the data immediately if 1) fetch request does not want to wait - // 2) fetch request does not require any data - // 3) has enough data to respond - // 4) some error happens while reading data - if(fetchRequest.maxWait <= 0 || - fetchRequest.numPartitions <= 0 || - bytesReadable >= fetchRequest.minBytes || - errorReadingData) { - debug("Returning fetch response %s for fetch request with correlation id %d to client %s" - .format(dataRead.values.map(_.data.error).mkString(","), fetchRequest.correlationId, fetchRequest.clientId)) - val response = new FetchResponse(fetchRequest.correlationId, dataRead.mapValues(_.data)) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) - } else { - debug("Putting fetch request with correlation id %d from client %s into purgatory".format(fetchRequest.correlationId, - fetchRequest.clientId)) - // create a list of (topic, partition) pairs to use as keys for this delayed request - val delayedFetchKeys = fetchRequest.requestInfo.keys.toSeq.map(new TopicPartitionRequestKey(_)) - - //val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest.maxWait, fetchRequest, - // dataRead.mapValues(_.offset)) - - def callback(succeeded: Boolean) { - if (succeeded) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(FetchResponse(fetch.correlationId, replicaManager.readMessageSets(fetchRequest).mapValues(_.data))))) - } - - val delayedFetch = new DelayedFetch( - fetchRequest.maxWait, - callback, - dataRead.mapValues(_.offset), - replicaManager) + // the callback for sending the response + def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { - // add the fetch request for watch if it's not satisfied, otherwise send the response back - val satisfiedByMe = fetchRequestPurgatory.checkAndMaybeWatch(delayedFetch) - if (satisfiedByMe) - fetchRequestPurgatory.respond(delayedFetch) + val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } - } - private def recordFollowerLogEndOffsets(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { - debug("Record follower log end offsets: %s ".format(offsets)) - offsets.foreach { - case (topicAndPartition, offset) => - replicaManager.updateReplicaLEOAndPartitionHW(topicAndPartition.topic, - topicAndPartition.partition, replicaId, offset) - - // for producer requests with ack > 1, we need to check - // if they can be unblocked after some follower's log end offsets have moved - replicaManager.unblockDelayedProduceRequests(new TopicPartitionRequestKey(topicAndPartition)) - } + // call the replica manager to append messages to the replicas + replicaManager.fetchMessages( + fetchRequest.maxWait.toLong, + fetchRequest.replicaId, + fetchRequest.minBytes, + fetchRequest.requestInfo, + sendResponseCallback) } /** diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 4abf034..18e2ea3 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -20,11 +20,11 @@ import kafka.api._ import kafka.common._ import kafka.utils._ import kafka.cluster.{Broker, Partition, Replica} -import kafka.log.LogManager +import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.common.TopicAndPartition -import kafka.message.MessageSet +import kafka.message.{ByteBufferMessageSet, MessageSet} import java.util.concurrent.atomic.AtomicBoolean import java.io.{IOException, File} @@ -45,6 +45,19 @@ object ReplicaManager { case class PartitionDataAndOffset(data: FetchResponsePartitionData, offset: LogOffsetMetadata) +case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} + +case class LogReadResult(info: FetchDataInfo, hw: Long, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} class ReplicaManager(config: KafkaConfig, time: Time, @@ -237,74 +250,228 @@ class ReplicaManager(config: KafkaConfig, } /** - * Read from all the offset details given and return a map of - * (topic, partition) -> PartitionData + * Append messages to leader replicas of the partition, and wait for replicated to other replicas, + * the callback function will be triggered either when timeout or the required acks are satisfied */ - def readMessageSets(fetchRequest: FetchRequest) = { - val isFetchFromFollower = fetchRequest.isFromFollower - fetchRequest.requestInfo.map - { - case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => - val partitionDataAndOffsetInfo = - try { - val (fetchInfo, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId) - BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes) - if (isFetchFromFollower) { - debug("Partition [%s,%d] received fetch request from follower %d" - .format(topic, partition, fetchRequest.replicaId)) - } - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, fetchInfo.messageSet), fetchInfo.fetchOffset) - } catch { - // NOTE: Failed fetch requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException - // since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request - // for a partition it is the leader for - case utpe: UnknownTopicOrPartitionException => - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, utpe.getMessage)) - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - case nle: NotLeaderForPartitionException => - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage)) - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - case t: Throwable => - BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() - error("Error when processing fetch request for partition [%s,%d] offset %d from %s with correlation id %d. Possible cause: %s" - .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId, t.getMessage)) - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - } - (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) + def appendMessages(timeout: Long, + requiredAcks : Short, + messagesPerPartition: Map[TopicAndPartition, MessageSet], + callbackOnComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { + + val sTime = SystemTime.milliseconds + val localProduceResults = appendToLocalLog(messagesPerPartition) + debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) + + val produceStatus = localProduceResults.mapValues(result => + ProduceStatus( + result.info.lastOffset + 1 // required offset + ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status + ) + + if(requiredAcks == 0) { + // if required acks = 0 we can trigger complete immediately + val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) + callbackOnComplete(produceResponseStatus) + } else if (produceRequest.requiredAcks == 1 || + messagesPerPartition.size <= 0 || + localProduceResults.values.count(_.error.isDefined) == produceRequest.numPartitions) { + // if required acks = 1 or all partition appends have failed we can trigger complete immediately + val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) + callbackOnComplete(produceResponseStatus) + } else { + // create delayed produce operation and try to watch it in the purgatory + val delayedRequest = new DelayedProduce(timeout, ProduceInfo(requiredAcks, produceStatus), this, callbackOnComplete) + val producerRequestKeys = messagesPerPartition.keys.map(TopicPartitionRequestKey(_)).toSeq + + val completedByMe = producerRequestPurgatory.tryCompleteElseWatch(delayedRequest, producerRequestKeys) + if (completedByMe) { + val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) + callbackOnComplete(produceResponseStatus) + } + } + } + + /** + * Append the messages to the local replica logs + */ + private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]): Map[TopicAndPartition, LogAppendResult] = { + trace("Append [%s] to local log ".format(messagesPerPartition)) + messagesPerPartition.map { case (topicAndPartition, messages) => + try { + val partitionOpt = getPartition(topicAndPartition.topic, topicAndPartition.partition) + val info = partitionOpt match { + case Some(partition) => + partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet]) + case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" + .format(topicAndPartition, brokerId)) + } + + val numAppendedMessages = + if (info.firstOffset == -1L || info.lastOffset == -1L) + 0 + else + info.lastOffset - info.firstOffset + 1 + + // update stats for successfully appended bytes and messages as bytesInRate and messageInRate + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) + BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) + + trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" + .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) + (topicAndPartition, LogAppendResult(info)) + } catch { + // NOTE: Failed produce requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException + // since failed produce requests metric is supposed to indicate failure of a broker in handling a produce request + // for a partition it is the leader for + case e: KafkaStorageException => + fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) + Runtime.getRuntime.halt(1) + (topicAndPartition, null) + case utpe: UnknownTopicOrPartitionException => // TODO + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + producerRequest.correlationId, producerRequest.clientId, topicAndPartition, utpe.getMessage)) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, utpe)) + case nle: NotLeaderForPartitionException => + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage)) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, nle)) + case e: Throwable => + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() + error("Error processing ProducerRequest with correlation id %d from client %s on partition %s" + .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, e)) + } + } + } + + /** + * Fetch messages from the leader replica, + * the callback function will be triggered either when timeout or required fetch info is satisfied + */ + def fetchMessages(timeout: Long, + replicaId: Int, + fetchMinBytes: Int, + fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], + callbackOnComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + + val fetchOnlyLeader: Boolean = replicaId != Request.DebuggingConsumerId + val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(fetchRequest.replicaId) + + // read from local logs + val fetchResults = readFromLocalLog(fetchOnlyLeader, fetchOnlyCommitted, fetchInfo) + + // if the fetch comes from the follower, + // update its corresponding log end offset + if(Request.isValidBrokerId(fetchRequest.replicaId)) + recordFollowerLogEndOffsets(replicaId, dataRead.mapValues(_.offset)) + + // check if this fetch request can be satisfied right away + val bytesReadable = fetchResults.values.map(_.info.messageSet.sizeInBytes).sum + val errorReadingData = fetchResults.values.foldLeft(false) ((errorIncurred, readResult) => + errorIncurred || (readResult.errorCode != ErrorMapping.NoError)) + // send the data immediately if 1) fetch request does not want to wait + // 2) fetch request does not require any data + // 3) has enough data to respond + // 4) some error happens while reading data + if(timeout <= 0 || + fetchInfo.size <= 0 || + bytesReadable >= fetchMinBytes || + errorReadingData) { + val fetchPartitionData = fetchResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + callbackOnComplete(fetchPartitionData) + } else { + val fetchStartOffsets = fetchResults.mapValues(result => result.info.fetchOffset) + val delayedFetch = new DelayedFetch(time, FetchInfo(fetchMinBytes, fetchOnlyLeader, fetchOnlyCommitted, fetchStartOffsets), this, callbackOnComplete) + // create a list of (topic, partition) pairs to use as keys for this delayed request + val delayedFetchKeys = fetchInfo.keys.map(new TopicPartitionRequestKey(_)).toSeq + + // add the fetch request for watch if it's not satisfied, otherwise send the response back + val completedByMe = fetchRequestPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) + if (completedByMe) { + // fetch again to get whatever is available + val fetchPartitionData = readFromLocalLog(fetchOnlyLeader, fetchOnlyCommitted, fetchInfo) + .mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + callbackOnComplete(fetchPartitionData) + } } } /** * Read from a single topic/partition at the given offset upto maxSize bytes */ - private def readMessageSet(topic: String, - partition: Int, - offset: Long, - maxSize: Int, - fromReplicaId: Int): (FetchDataInfo, Long) = { - // check if the current broker is the leader for the partitions - val localReplica = if(fromReplicaId == Request.DebuggingConsumerId) - getReplicaOrException(topic, partition) - else - getLeaderReplicaIfLocal(topic, partition) - trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize)) - val maxOffsetOpt = - if (Request.isValidBrokerId(fromReplicaId)) - None - else - Some(localReplica.highWatermark.messageOffset) - val fetchInfo = localReplica.log match { - case Some(log) => - log.read(offset, maxSize, maxOffsetOpt) - case None => - error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) - FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) + private def readFromLocalLog(readOnlyIfLeader: Boolean, + readOnlyCommitted: Boolean, + readInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { + + readInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => + val partitionDataAndOffsetInfo = + try { + trace("Fetching log segment for topic %s, partition %d, offset %ld, size %d" + .format(topic, partition, offset, fetchSize)) + + // decide whether to only fetch from leader + val localReplica = if (readOnlyIfLeader) + getLeaderReplicaIfLocal(topic, partition) + else + getReplicaOrException(topic, partition) + + // decide whether to only fetch committed data (i.e. messages below high watermark) + val maxOffsetOpt = if (readOnlyCommitted) + None + else + Some(localReplica.highWatermark.messageOffset) + + // read on log + val logReadInfo = localReplica.log match { + case Some(log) => + log.read(offset, maxSize, maxOffsetOpt) + case None => + error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) + FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) + } + + BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) + + LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, ErrorMapping.NoError) + } catch { + // NOTE: Failed fetch requests is not incremented for UnknownTopicOrPartitionException, NotLeaderForPartitionException + // and ReplicaNotAvailableException since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request + // for a partition it is the leader for + case utpe: UnknownTopicOrPartitionException => // TODO + warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( + fetchRequest.correlationId, fetchRequest.clientId, topic, partition, utpe.getMessage)) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, utpe) + case nle: NotLeaderForPartitionException => + warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( + fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage)) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, nle) + case rnae: ReplicaNotAvailableException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, rnae) + case t: Throwable => + BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() + error("Error when processing fetch request for partition [%s,%d] offset %d from %s with correlation id %d. Possible cause: %s" + .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId, t.getMessage)) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, t) + } + (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) + } + } + + private def recordFollowerLogEndOffsets(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { + debug("Record follower log end offsets: %s ".format(offsets)) + offsets.foreach { + case (topicAndPartition, offset) => + updateReplicaLEOAndPartitionHW(topicAndPartition.topic, topicAndPartition.partition, replicaId, offset) + + // for producer requests with ack > 1, we need to check + // if they can be unblocked after some follower's log end offsets have moved + unblockDelayedProduceRequests(new TopicPartitionRequestKey(topicAndPartition)) } - (fetchInfo, localReplica.highWatermark.messageOffset) } def maybeUpdateMetadataCache(updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index c55aac1..717965c 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -34,7 +34,7 @@ import com.yammer.metrics.core.Gauge * message append operation could be waiting for specified number of acks; or a delayed * message fetch operation could be waiting for a given number of bytes to accumulate. */ -abstract class DelayedRequest(delayMs: Long, onComplete: Boolean => Unit) extends DelayedItem(delayMs) { +abstract class DelayedRequest(delayMs: Long, onComplete: Any => Unit) extends DelayedItem(delayMs) { val completed = new AtomicBoolean(false) /* @@ -48,7 +48,7 @@ abstract class DelayedRequest(delayMs: Long, onComplete: Boolean => Unit) extend /* * When delayMs has elapsed, expire the delayed operation */ - def onExpired() = onComplete(false) + def onExpired() = onComplete(null) } /** @@ -97,20 +97,25 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt */ def tryCompleteElseWatch(operation: DelayedRequest, watchKeys: Seq[Any]): Boolean = { for(key <- watchKeys) { + // if the operation is already completed, stopping adding it to + // any further lists and return false + if (operation.completed.get()) + return false val watchers = watchersFor(key) - // if the operation is found completed, stop adding it to any further - // lists and return true immediately - if(!watchers.checkAndMaybeAdd(operation)) { + // if the operation is completed by myself, stop adding it to + // any further lists and return true immediately + if(! watchers.checkAndMaybeAdd(operation)) { return true } } // if it is indeed watched, add to the expire queue also - watched.getAndIncrement() - expirationReaper.enqueue(operation) + if (! operation.completed.get()) { + watched.getAndIncrement() + expirationReaper.enqueue(operation) + } false - } /** @@ -154,9 +159,6 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt // potentially add the element to watch if it is not satisfied yet def checkAndMaybeAdd(t: T): Boolean = { synchronized { - // if it is already satisfied, return false - if (t.completed.get()) - return false // if the operation can be completed, return false; otherwise add to watch list if(t.tryComplete()) { return false -- 1.7.12.4 From e4cbc4945dae3a45b864efa7bbe23eb3638996b1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 13 Aug 2014 16:08:14 -0700 Subject: [PATCH 05/33] removal respond from delayed requests --- core/src/main/scala/kafka/server/DelayedFetch.scala | 5 ----- core/src/main/scala/kafka/server/DelayedProduce.scala | 17 ----------------- 2 files changed, 22 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 4b7542f..40abebe 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -101,9 +101,4 @@ class DelayedFetch(delayMs: Long, val readData = replicaManager.readMessageSets(fetch) onComplete(readData) } - - def respond(replicaManager: ReplicaManager): FetchResponse = { - val topicData = replicaManager.readMessageSets(fetch) - FetchResponse(fetch.correlationId, topicData.mapValues(_.data)) - } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 99bdf6f..1b6ad16 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -106,22 +106,5 @@ class DelayedProduce(delayMs: Long, val responseStatus = produceInfo.produceStatus.mapValues(status => status.responseStatus) onComplete(responseStatus) } - - def respond(offsetManager: OffsetManager): RequestOrResponse = { - val responseStatus = partitionStatus.mapValues(status => status.responseStatus) - - val errorCode = responseStatus.find { case (_, status) => - status.error != ErrorMapping.NoError - }.map(_._2.error).getOrElse(ErrorMapping.NoError) - - if (errorCode == ErrorMapping.NoError) { - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) - } - - val response = offsetCommitRequestOpt.map(_.responseFor(errorCode, offsetManager.config.maxMetadataSize)) - .getOrElse(ProducerResponse(produce.correlationId, responseStatus)) - - response - } } -- 1.7.12.4 From bff3a21a51b455414849f23e3e63a0ec2d6077af Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 14 Aug 2014 16:36:35 -0700 Subject: [PATCH 06/33] move warning logs --- .../src/main/scala/kafka/common/ErrorMapping.scala | 2 ++ core/src/main/scala/kafka/server/KafkaApis.scala | 21 +++++++++++++-- .../main/scala/kafka/server/ReplicaManager.scala | 30 +++++++--------------- 3 files changed, 30 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index 880ab4a..eedc2f5 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -84,4 +84,6 @@ object ErrorMapping { throw codeToException(code).newInstance() def exceptionFor(code: Short) : Throwable = codeToException(code).newInstance() + + def exceptionNameFor(code: Short) : String = codeToException(code).getName() } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 64d190c..65084b8 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -155,13 +155,22 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending the response def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { - val numPartitionsInError = responseStatus.values.count(_.error.isDefined) + var errorInResponse = false + responseStatus.foreach { case (topicAndPartition, status) => + // Here we only print warnings for known errors; if it is unknown, it will cause + // an error message in the replica manager already and hence can be ignored here + if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + produceRequest.correlationId, produceRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) + errorInResponse = true + } + } if(produceRequest.requiredAcks == 0) { // no operation needed if producer request.required.acks = 0; however, if there is any exception in handling the request, since // no response is expected by the producer the handler will send a close connection response to the socket server // to close the socket so that the producer client will know that some exception has happened and will refresh its metadata - if (numPartitionsInError != 0) { + if (errorInResponse) { info(("Send the close connection response due to error handling produce request " + "[clientId = %s, correlationId = %s, topicAndPartition = %s] with Ack=0") .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) @@ -194,6 +203,14 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending the response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + responsePartitionData.foreach { case (topicAndPartition, response) => + // Here we only print warnings for known errors; if it is unknown, it will cause + // an error message in the replica manager already and hence can be ignored here + if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { + warn("Fetch request with correlation id %d from client %s on partition %s failed due to %s".format( + fetchRequest.correlationId, fetchRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(response.error))) + } + } val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 18e2ea3..81f7a7c 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -264,7 +264,7 @@ class ReplicaManager(config: KafkaConfig, val produceStatus = localProduceResults.mapValues(result => ProduceStatus( - result.info.lastOffset + 1 // required offset + result.info.lastOffset + 1, // required offset ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status ) @@ -322,26 +322,20 @@ class ReplicaManager(config: KafkaConfig, .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) (topicAndPartition, LogAppendResult(info)) } catch { - // NOTE: Failed produce requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException - // since failed produce requests metric is supposed to indicate failure of a broker in handling a produce request - // for a partition it is the leader for + // NOTE: Failed produce requests metric is not incremented for known exceptions + // it is supposed to indicate un-expected failures of a broker in handling a produce request case e: KafkaStorageException => fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) Runtime.getRuntime.halt(1) (topicAndPartition, null) case utpe: UnknownTopicOrPartitionException => // TODO - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, utpe.getMessage)) (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, utpe)) case nle: NotLeaderForPartitionException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage)) (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, nle)) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() - error("Error processing ProducerRequest with correlation id %d from client %s on partition %s" - .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e) + error("Error processing append operation on partition %s".format(topicAndPartition), e) (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, e)) } } @@ -409,7 +403,7 @@ class ReplicaManager(config: KafkaConfig, readInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => val partitionDataAndOffsetInfo = try { - trace("Fetching log segment for topic %s, partition %d, offset %ld, size %d" + trace("Fetching log segment for topic %s, partition %d, offset %d, size %d" .format(topic, partition, offset, fetchSize)) // decide whether to only fetch from leader @@ -438,24 +432,18 @@ class ReplicaManager(config: KafkaConfig, LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, ErrorMapping.NoError) } catch { - // NOTE: Failed fetch requests is not incremented for UnknownTopicOrPartitionException, NotLeaderForPartitionException - // and ReplicaNotAvailableException since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request - // for a partition it is the leader for + // NOTE: Failed fetch requests metric is not incremented for known exceptions since it + // is supposed to indicate un-expected failure of a broker in handling a fetch request case utpe: UnknownTopicOrPartitionException => // TODO - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, utpe.getMessage)) LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, utpe) case nle: NotLeaderForPartitionException => - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage)) LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, nle) case rnae: ReplicaNotAvailableException => LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, rnae) - case t: Throwable => + case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() - error("Error when processing fetch request for partition [%s,%d] offset %d from %s with correlation id %d. Possible cause: %s" - .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId, t.getMessage)) + error("Error processing fetch operation on partition [%s,%d] offset %d".format(topic, partition, offset)) LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, t) } (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) -- 1.7.12.4 From 1419b135fa70a70f1c3d0114eab4d7e9d4d66402 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 16 Aug 2014 11:13:34 -0700 Subject: [PATCH 07/33] dummy --- core/src/main/scala/kafka/server/KafkaApis.scala | 10 +++++++--- core/src/main/scala/kafka/server/ReplicaManager.scala | 4 ++-- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 65084b8..0ac8ec4 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -56,7 +56,7 @@ class KafkaApis(val requestChannel: RequestChannel, try{ trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) request.requestId match { - case RequestKeys.ProduceKey => handleProducerOrOffsetCommitRequest(request) + case RequestKeys.ProduceKey => handleProducerRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) case RequestKeys.OffsetsKey => handleOffsetRequest(request) case RequestKeys.MetadataKey => handleTopicMetadataRequest(request) @@ -64,7 +64,7 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request) case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request) case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request) - case RequestKeys.OffsetCommitKey => handleProducerOrOffsetCommitRequest(request) + case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) @@ -123,6 +123,10 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) } + def handleOffsetCommitRequest(request: RequestChannel.Request) { + + } + private def producerRequestFromOffsetCommit(offsetCommitRequest: OffsetCommitRequest) = { val msgs = offsetCommitRequest.filterLargeMetadata(config.offsetMetadataMaxSize).map { case (topicAndPartition, offset) => @@ -150,7 +154,7 @@ class KafkaApis(val requestChannel: RequestChannel, /** * Handle a produce request or offset commit request (which is really a specialized producer request) */ - def handleProducerOrOffsetCommitRequest(request: RequestChannel.Request) { + def handleProducerRequest(request: RequestChannel.Request) { val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] // the callback for sending the response diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 81f7a7c..e046166 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -221,7 +221,7 @@ class ReplicaManager(config: KafkaConfig, def getReplicaOrException(topic: String, partition: Int): Replica = { val replicaOpt = getReplica(topic, partition) if(replicaOpt.isDefined) - return replicaOpt.get + replicaOpt.get else throw new ReplicaNotAvailableException("Replica %d is not available for partition [%s,%d]".format(config.brokerId, topic, partition)) } @@ -434,7 +434,7 @@ class ReplicaManager(config: KafkaConfig, } catch { // NOTE: Failed fetch requests metric is not incremented for known exceptions since it // is supposed to indicate un-expected failure of a broker in handling a fetch request - case utpe: UnknownTopicOrPartitionException => // TODO + case utpe: UnknownTopicOrPartitionException => LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, utpe) case nle: NotLeaderForPartitionException => LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, nle) -- 1.7.12.4 From 73fa0b36d5cec90553cb5f0c63005e6b9b5e7481 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 18 Aug 2014 10:40:04 -0700 Subject: [PATCH 08/33] dummy --- core/src/main/scala/kafka/api/FetchRequest.scala | 1 - .../src/main/scala/kafka/server/DelayedFetch.scala | 50 +++++++++------- .../main/scala/kafka/server/DelayedProduce.scala | 30 +++++----- core/src/main/scala/kafka/server/KafkaApis.scala | 39 +++++++++---- .../main/scala/kafka/server/ReplicaManager.scala | 66 +++++++++++----------- .../main/scala/kafka/server/RequestPurgatory.scala | 35 ++++++------ 6 files changed, 123 insertions(+), 98 deletions(-) diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 59c0915..b038c15 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -30,7 +30,6 @@ import scala.collection.immutable.Map case class PartitionFetchInfo(offset: Long, fetchSize: Int) - object FetchRequest { val CurrentVersion = 0.shortValue val DefaultMaxWait = 0 diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 40abebe..c80ed17 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -17,12 +17,10 @@ package kafka.server -import kafka.network.RequestChannel -import kafka.api.{FetchResponse, FetchRequest} +import kafka.api.{FetchResponsePartitionData, PartitionFetchInfo, FetchResponse, FetchRequest} import kafka.common.{UnknownTopicOrPartitionException, NotLeaderForPartitionException, TopicAndPartition} import scala.collection.immutable.Map -import scala.collection.Seq /** * A delayed fetch request, which is satisfied (or more @@ -37,43 +35,49 @@ import scala.collection.Seq * - should return whatever data is available. */ -case class FetchInfo(fetchMinBytes: Int, - fetchOnlyLeader: Boolean, - fetchOnlyCommitted: Boolean, - fetchStartOffsets: Map[TopicAndPartition, LogOffsetMetadata]) { +case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInfo: PartitionFetchInfo) { - override def toString = "FetchInfo [minBytes: " + fetchMinBytes + "] : " + - "[committedOnly: " + fetchOnlyCommitted + "] : " - "[startOffsets: " + fetchStartOffsets + "]" + override def toString = "[startOffset: " + startOffsetMetadata + ", " + + "fetchSize: " + fetchSize + "]" +} + +case class FetchMetadata(fetchMinBytes: Int, + fetchOnlyLeader: Boolean, + fetchOnlyCommitted: Boolean, + fetchPartitionStatus: Map[TopicAndPartition, FetchPartitionStatus]) { + + override def toString = "[minBytes: " + fetchMinBytes + ", " + + "committedOnly: " + fetchOnlyCommitted + ", " + "partitionStatus: " + fetchPartitionStatus + "]" } class DelayedFetch(delayMs: Long, - fetchInfo: FetchInfo, + fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, onComplete: Map[TopicAndPartition, PartitionDataAndOffset] => Unit) extends DelayedRequest(delayMs, onComplete) { override def tryComplete() : Boolean = { var accumulatedSize = 0 - fetchInfo.fetchStartOffsets.foreach { + fetchMetadata.fetchPartitionStatus.foreach { case (topicAndPartition, fetchOffset) => try { if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) { val replica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) val endOffset = - if (fetchInfo.fetchOnlyCommitted) + if (fetchMetadata.fetchOnlyCommitted) replica.highWatermark else replica.logEndOffset if (endOffset.offsetOnOlderSegment(fetchOffset)) { // Case C, this can happen when the new follower replica fetching on a truncated leader - debug("Satisfying %s since it is fetching later segments of partition %s.".format(fetchInfo, topicAndPartition)) + debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) return super.tryComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { // Case C, this can happen when the folloer replica is lagging too much - debug("Satisfying %s immediately since it is fetching older segments.".format(fetchInfo)) + debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata)) return super.tryComplete() } else if (fetchOffset.precedes(endOffset)) { accumulatedSize += endOffset.positionDiff(fetchOffset) @@ -81,24 +85,30 @@ class DelayedFetch(delayMs: Long, } } catch { case utpe: UnknownTopicOrPartitionException => // Case A - debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchInfo)) + debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) return super.tryComplete() case nle: NotLeaderForPartitionException => // Case B - debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchInfo)) + debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) return super.tryComplete() } } // Case D - if (accumulatedSize >= fetchInfo.fetchMinBytes) + if (accumulatedSize >= fetchMetadata.fetchMinBytes) super.tryComplete() else false } override def onExpired() { + debug("Expire fetch %s and return whatever fetch data is available".format(fetchMetadata)) + // read whatever data is available and return - val readData = replicaManager.readMessageSets(fetch) - onComplete(readData) + val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, + fetchMetadata.fetchOnlyCommitted, + fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) + val fetchPartitionData = logReadResults.mapValues(result => + FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + onComplete(fetchPartitionData) } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 1b6ad16..0ec0055 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -21,11 +21,9 @@ import kafka.api._ import kafka.common.ErrorMapping import kafka.common.TopicAndPartition import kafka.utils.Logging -import kafka.network.RequestChannel import scala.Some import scala.collection.immutable.Map -import scala.collection.Seq /** A delayed produce request, which is satisfied (or more * accurately, unblocked) -- if for every partition it produce to: @@ -35,28 +33,28 @@ import scala.collection.Seq * B.2 - else, at least requiredAcks replicas should be caught up to this request. */ -case class ProduceStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { +case class ProducePartitionStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { @volatile var acksPending = false - override def toString = "acksPending:%b, error: %d, startOffset: %d, requiredOffset: %d" + override def toString = "[acksPending: %b, error: %d, startOffset: %d, requiredOffset: %d]" .format(acksPending, responseStatus.error, responseStatus.offset, requiredOffset) } -case class ProduceInfo(produceRequiredAcks: Short, - produceStatus: Map[TopicAndPartition, ProduceStatus]) { +case class ProduceMetadata(produceRequiredAcks: Short, + produceStatus: Map[TopicAndPartition, ProducePartitionStatus]) { - override def toString = "ProduceInfo [requiredBytes: " + fetchMinBytes + "] : " + - "[partitionStatus: " + produceStatus + "]" + override def toString = "[requiredAcks: %d, partitionStatus: %s]" + .format(produceRequiredAcks, produceStatus) } class DelayedProduce(delayMs: Long, - produceInfo: ProduceInfo, + produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, onComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) extends DelayedRequest(delayMs) with Logging { // first update the acks pending variable according to the error code - produceInfo.produceStatus foreach { case (topicAndPartition, status) => + produceMetadata.produceStatus foreach { case (topicAndPartition, status) => if (status.responseStatus.error == ErrorMapping.NoError) { // Timeout error state will be cleared when required acks are received status.acksPending = true @@ -70,8 +68,8 @@ class DelayedProduce(delayMs: Long, def tryComplete(): Boolean = { // check for each partition if it still has pending acks - produceInfo.produceStatus.foreach { case (topicAndPartition, status) => - trace("Checking producer request satisfaction for %s, acksPending = %b" + produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => + trace("Checking produce satisfaction for %s, acksPending = %b" .format(topicAndPartition, status.acksPending)) // skip those partitions that have already been satisfied if (status.acksPending) { @@ -80,7 +78,7 @@ class DelayedProduce(delayMs: Long, case Some(partition) => partition.checkEnoughReplicasReachOffset( status.requiredOffset, - produceInfo.produceRequiredAcks) + produceMetadata.produceRequiredAcks) case None => (false, ErrorMapping.UnknownTopicOrPartitionCode) } @@ -95,15 +93,17 @@ class DelayedProduce(delayMs: Long, } // unblocked if there are no partitions with pending acks - if (! produceInfo.produceStatus.values.exists(p => p.acksPending)) + if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) super.tryComplete() else false } override def onExpired() { + debug("Expire produce %s and return the error codes".format(produceMetadata)) + // return the current response status - val responseStatus = produceInfo.produceStatus.mapValues(status => status.responseStatus) + val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) onComplete(responseStatus) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 0ac8ec4..04b5d5f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -124,7 +124,21 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleOffsetCommitRequest(request: RequestChannel.Request) { + val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + // the callback for sending the response + def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + var errorInResponse = false + responseStatus.foreach { case (topicAndPartition, status) => + // Here we only print warnings for known errors; if it is unknown, it will cause + // an error message in the replica manager already and hence can be ignored here + if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s" + .format(produceRequest.correlationId, produceRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) + errorInResponse = true + } + } } private def producerRequestFromOffsetCommit(offsetCommitRequest: OffsetCommitRequest) = { @@ -164,20 +178,20 @@ class KafkaApis(val requestChannel: RequestChannel, // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - produceRequest.correlationId, produceRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s" + .format(produceRequest.correlationId, produceRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) errorInResponse = true } } if(produceRequest.requiredAcks == 0) { - // no operation needed if producer request.required.acks = 0; however, if there is any exception in handling the request, since - // no response is expected by the producer the handler will send a close connection response to the socket server - // to close the socket so that the producer client will know that some exception has happened and will refresh its metadata + // no operation needed if producer request.required.acks = 0; however, if there is any error in handling + // the request, since no response is expected by the producer, the server will close socket server so that + // the producer client will know that some error has happened and will refresh its metadata if (errorInResponse) { - info(("Send the close connection response due to error handling produce request " + - "[clientId = %s, correlationId = %s, topicAndPartition = %s] with Ack=0") - .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) + info("Close connection due to error handling produce request with correlation id %d from client id %s with ack=0" + .format(produceRequest.correlationId, produceRequest.clientId)) requestChannel.closeConnection(request.processor, request) } else { requestChannel.noOperation(request.processor, request) @@ -207,12 +221,13 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending the response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { - responsePartitionData.foreach { case (topicAndPartition, response) => + responsePartitionData.foreach { case (topicAndPartition, data) => // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here - if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { - warn("Fetch request with correlation id %d from client %s on partition %s failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(response.error))) + if (data.error != ErrorMapping.NoError && data.error != ErrorMapping.UnknownCode) { + warn("Fetch request with correlation id %d from client %s on partition %s failed due to %s" + .format(fetchRequest.correlationId, fetchRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(data.error))) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e046166..8b55063 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -52,7 +52,7 @@ case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) } } -case class LogReadResult(info: FetchDataInfo, hw: Long, error: Option[Throwable] = None) { +case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Option[Throwable] = None) { def errorCode = error match { case None => ErrorMapping.NoError case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) @@ -254,7 +254,7 @@ class ReplicaManager(config: KafkaConfig, * the callback function will be triggered either when timeout or the required acks are satisfied */ def appendMessages(timeout: Long, - requiredAcks : Short, + requiredAcks: Short, messagesPerPartition: Map[TopicAndPartition, MessageSet], callbackOnComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { @@ -263,7 +263,7 @@ class ReplicaManager(config: KafkaConfig, debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) val produceStatus = localProduceResults.mapValues(result => - ProduceStatus( + ProducePartitionStatus( result.info.lastOffset + 1, // required offset ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status ) @@ -280,10 +280,11 @@ class ReplicaManager(config: KafkaConfig, callbackOnComplete(produceResponseStatus) } else { // create delayed produce operation and try to watch it in the purgatory - val delayedRequest = new DelayedProduce(timeout, ProduceInfo(requiredAcks, produceStatus), this, callbackOnComplete) + val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) + val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, callbackOnComplete) val producerRequestKeys = messagesPerPartition.keys.map(TopicPartitionRequestKey(_)).toSeq - val completedByMe = producerRequestPurgatory.tryCompleteElseWatch(delayedRequest, producerRequestKeys) + val completedByMe = producerRequestPurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) if (completedByMe) { val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) callbackOnComplete(produceResponseStatus) @@ -294,7 +295,7 @@ class ReplicaManager(config: KafkaConfig, /** * Append the messages to the local replica logs */ - private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]): Map[TopicAndPartition, LogAppendResult] = { + private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]) = { trace("Append [%s] to local log ".format(messagesPerPartition)) messagesPerPartition.map { case (topicAndPartition, messages) => try { @@ -303,7 +304,7 @@ class ReplicaManager(config: KafkaConfig, case Some(partition) => partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet]) case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" - .format(topicAndPartition, brokerId)) + .format(topicAndPartition, localBrokerId)) } val numAppendedMessages = @@ -328,7 +329,7 @@ class ReplicaManager(config: KafkaConfig, fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) Runtime.getRuntime.halt(1) (topicAndPartition, null) - case utpe: UnknownTopicOrPartitionException => // TODO + case utpe: UnknownTopicOrPartitionException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, utpe)) case nle: NotLeaderForPartitionException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, nle)) @@ -355,7 +356,7 @@ class ReplicaManager(config: KafkaConfig, val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(fetchRequest.replicaId) // read from local logs - val fetchResults = readFromLocalLog(fetchOnlyLeader, fetchOnlyCommitted, fetchInfo) + val logReadResults = readFromLocalLog(fetchOnlyLeader, fetchOnlyCommitted, fetchInfo) // if the fetch comes from the follower, // update its corresponding log end offset @@ -363,24 +364,26 @@ class ReplicaManager(config: KafkaConfig, recordFollowerLogEndOffsets(replicaId, dataRead.mapValues(_.offset)) // check if this fetch request can be satisfied right away - val bytesReadable = fetchResults.values.map(_.info.messageSet.sizeInBytes).sum - val errorReadingData = fetchResults.values.foldLeft(false) ((errorIncurred, readResult) => + val bytesReadable = logReadResults.values.map(_.info.messageSet.sizeInBytes).sum + val errorReadingData = logReadResults.values.foldLeft(false) ((errorIncurred, readResult) => errorIncurred || (readResult.errorCode != ErrorMapping.NoError)) + // send the data immediately if 1) fetch request does not want to wait // 2) fetch request does not require any data // 3) has enough data to respond // 4) some error happens while reading data - if(timeout <= 0 || - fetchInfo.size <= 0 || - bytesReadable >= fetchMinBytes || - errorReadingData) { - val fetchPartitionData = fetchResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + if(timeout <= 0 || fetchInfo.size <= 0 || bytesReadable >= fetchMinBytes || errorReadingData) { + val fetchPartitionData = logReadResults.mapValues(result => + FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) callbackOnComplete(fetchPartitionData) } else { - val fetchStartOffsets = fetchResults.mapValues(result => result.info.fetchOffset) - val delayedFetch = new DelayedFetch(time, FetchInfo(fetchMinBytes, fetchOnlyLeader, fetchOnlyCommitted, fetchStartOffsets), this, callbackOnComplete) + // construct the fetch results from the read results + val fetchPartitionStatus = logReadResults.mapValues(result => FetchPartitionStatus(result.info.fetchOffset, result.readSize)) + val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyLeader, fetchOnlyCommitted, fetchPartitionStatus) + val delayedFetch = new DelayedFetch(time, fetchMetadata, this, callbackOnComplete) + // create a list of (topic, partition) pairs to use as keys for this delayed request - val delayedFetchKeys = fetchInfo.keys.map(new TopicPartitionRequestKey(_)).toSeq + val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionRequestKey(_)).toSeq // add the fetch request for watch if it's not satisfied, otherwise send the response back val completedByMe = fetchRequestPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) @@ -396,15 +399,14 @@ class ReplicaManager(config: KafkaConfig, /** * Read from a single topic/partition at the given offset upto maxSize bytes */ - private def readFromLocalLog(readOnlyIfLeader: Boolean, - readOnlyCommitted: Boolean, - readInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { + def readFromLocalLog(readOnlyIfLeader: Boolean, + readOnlyCommitted: Boolean, + readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]) = { - readInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => + readPartitionInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => val partitionDataAndOffsetInfo = try { - trace("Fetching log segment for topic %s, partition %d, offset %d, size %d" - .format(topic, partition, offset, fetchSize)) + trace("Fetching log segment for topic %s, partition %d, offset %d, size %d".format(topic, partition, offset, fetchSize)) // decide whether to only fetch from leader val localReplica = if (readOnlyIfLeader) @@ -428,23 +430,23 @@ class ReplicaManager(config: KafkaConfig, } BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) - LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, ErrorMapping.NoError) + LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, ErrorMapping.NoError) } catch { // NOTE: Failed fetch requests metric is not incremented for known exceptions since it // is supposed to indicate un-expected failure of a broker in handling a fetch request case utpe: UnknownTopicOrPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, utpe) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, utpe) case nle: NotLeaderForPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, nle) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, nle) case rnae: ReplicaNotAvailableException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, rnae) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, rnae) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() error("Error processing fetch operation on partition [%s,%d] offset %d".format(topic, partition, offset)) - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, t) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, t) } (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) } diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 717965c..ebfcd71 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -38,17 +38,24 @@ abstract class DelayedRequest(delayMs: Long, onComplete: Any => Unit) extends De val completed = new AtomicBoolean(false) /* - * Check if the delayed operation is already completed + * Check if the delayed operation can be completed * * Note that concurrent threads can check if an operation can be completed or not, - * but only the first thread will succeed in completing the operation + * but only the first thread will succeed in completing the operation and return + * true, others will still return false */ def tryComplete(): Boolean = completed.compareAndSet(false, true) + /** + * Check if the delayed operation is already completed + */ + def isCompleted(): Boolean = completed.get() + /* * When delayMs has elapsed, expire the delayed operation */ - def onExpired() = onComplete(null) + def onExpired(): Unit + } /** @@ -61,9 +68,6 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt /* a list of requests watching each key */ private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) - /* the number of requests being watched, duplicates added on different watchers are also counted */ - private val watched = new AtomicInteger(0) - /* background thread expiring requests that have been waiting too long */ private val expirationReaper = new ExpiredOperationReaper @@ -99,7 +103,7 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt for(key <- watchKeys) { // if the operation is already completed, stopping adding it to // any further lists and return false - if (operation.completed.get()) + if (operation.isCompleted()) return false val watchers = watchersFor(key) // if the operation is completed by myself, stop adding it to @@ -110,8 +114,7 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt } // if it is indeed watched, add to the expire queue also - if (! operation.completed.get()) { - watched.getAndIncrement() + if (! operation.isCompleted()) { expirationReaper.enqueue(operation) } @@ -176,7 +179,7 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt var purged = 0 while (iter.hasNext) { val curr = iter.next - if(curr.completed.get()) { + if(curr.isCompleted()) { iter.remove() purged += 1 } @@ -192,16 +195,14 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt val iter = requests.iterator() while(iter.hasNext) { val curr = iter.next - if (curr.completed.get()) { + if (curr.isCompleted()) { // another thread has completed this request, just remove it iter.remove() } else { val completed = curr.tryComplete() if(completed) { iter.remove() - watched.getAndDecrement() response += curr - expirationReaper.satisfyRequest() } } } @@ -230,7 +231,6 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt */ def enqueue(t: T) { delayed.add(t) - unsatisfied.incrementAndGet() } /** @@ -239,7 +239,7 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt private def pollExpired(): T = { while (true) { val curr = delayed.poll(200L, TimeUnit.MILLISECONDS) - if (curr == null) + if (curr == null.asInstanceOf[T]) return null.asInstanceOf[T] // try set the operation failed (and hence completed), if succeed return it; // otherwise try to get the next expired operation since this one has been completed by others @@ -260,7 +260,7 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt val iter = delayed.iterator() while (iter.hasNext) { val curr = iter.next() - if (curr.completed.get()) { + if (curr.isCompleted()) { iter.remove() purged += 1 } @@ -272,7 +272,7 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt override def doWork() { val curr = pollExpired() - if (curr != null) { + if (curr != null.asInstanceOf[T]) { curr.onExpired() } if (size() >= purgeInterval) { // see if we need to force a full purge @@ -284,5 +284,4 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt } } } - } -- 1.7.12.4 From a77431af154eba974f9eb56ed078520269beabdf Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 18 Aug 2014 14:57:03 -0700 Subject: [PATCH 09/33] tons of compilation errors --- .../main/scala/kafka/api/OffsetCommitRequest.scala | 24 ++---- .../src/main/scala/kafka/server/DelayedFetch.scala | 5 ++ .../main/scala/kafka/server/DelayedProduce.scala | 5 +- core/src/main/scala/kafka/server/KafkaApis.scala | 52 ++++--------- .../main/scala/kafka/server/OffsetManager.scala | 90 ++++++++++++++++++++-- .../main/scala/kafka/server/ReplicaManager.scala | 80 ++++++++----------- .../main/scala/kafka/server/RequestPurgatory.scala | 2 +- .../scala/unit/kafka/server/SimpleFetchTest.scala | 4 +- 8 files changed, 149 insertions(+), 113 deletions(-) diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 861a6cf..050615c 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -78,28 +78,12 @@ case class OffsetCommitRequest(groupId: String, groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID, consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID) extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) { + assert(versionId == 0 || versionId == 1, "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) - def filterLargeMetadata(maxMetadataSize: Int) = - requestInfo.filter(info => info._2.metadata == null || info._2.metadata.length <= maxMetadataSize) - - def responseFor(errorCode: Short, offsetMetadataMaxSize: Int) = { - val commitStatus = requestInfo.map {info => - (info._1, if (info._2.metadata != null && info._2.metadata.length > offsetMetadataMaxSize) - ErrorMapping.OffsetMetadataTooLargeCode - else if (errorCode == ErrorMapping.UnknownTopicOrPartitionCode) - ErrorMapping.ConsumerCoordinatorNotAvailableCode - else if (errorCode == ErrorMapping.NotLeaderForPartitionCode) - ErrorMapping.NotCoordinatorForConsumerCode - else - errorCode) - }.toMap - OffsetCommitResponse(commitStatus, correlationId) - } - def writeTo(buffer: ByteBuffer) { // Write envelope buffer.putShort(versionId) @@ -150,8 +134,10 @@ case class OffsetCommitRequest(groupId: String, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) - val errorResponse = responseFor(errorCode, Int.MaxValue) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + val commitStatus = requestInfo.mapValues(_ => errorCode) + val commitResponse = OffsetCommitResponse(commitStatus, correlationId) + + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(commitResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index c80ed17..5cff718 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -102,13 +102,18 @@ class DelayedFetch(delayMs: Long, override def onExpired() { debug("Expire fetch %s and return whatever fetch data is available".format(fetchMetadata)) + completeFetch() + } + def completeFetch() { // read whatever data is available and return val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, fetchMetadata.fetchOnlyCommitted, fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) + val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + onComplete(fetchPartitionData) } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 0ec0055..bbb176c 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -66,7 +66,7 @@ class DelayedProduce(delayMs: Long, trace("Initial partition status for %s is %s".format(topicAndPartition, status)) } - def tryComplete(): Boolean = { + override def tryComplete(): Boolean = { // check for each partition if it still has pending acks produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => trace("Checking produce satisfaction for %s, acksPending = %b" @@ -101,7 +101,10 @@ class DelayedProduce(delayMs: Long, override def onExpired() { debug("Expire produce %s and return the error codes".format(produceMetadata)) + completeProduce() + } + def completeProduce() { // return the current response status val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) onComplete(responseStatus) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 04b5d5f..98b4e35 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -42,12 +42,8 @@ class KafkaApis(val requestChannel: RequestChannel, val config: KafkaConfig, val controller: KafkaController) extends Logging { - val producerRequestPurgatory = new ProducerRequestPurgatory(replicaManager, offsetManager, requestChannel) - val fetchRequestPurgatory = new FetchRequestPurgatory(replicaManager, requestChannel) - // TODO: the following line will be removed in 0.9 - replicaManager.initWithRequestPurgatory(producerRequestPurgatory, fetchRequestPurgatory) - var metadataCache = new MetadataCache this.logIdent = "[KafkaApi-%d] ".format(brokerId) + val metadataCache = new MetadataCache /** * Top-level method that handles all requests and multiplexes to the right api @@ -127,42 +123,28 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] // the callback for sending the response - def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { - var errorInResponse = false - responseStatus.foreach { case (topicAndPartition, status) => + def sendResponseCallback(responseStatus: Map[TopicAndPartition, Short]) { + responseStatus.foreach { case (topicAndPartition, errorCode) => // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here - if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s" - .format(produceRequest.correlationId, produceRequest.clientId, - topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) - errorInResponse = true + if (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.UnknownCode) { + warn("Offset commit request with correlation id %d from client %s on partition %s failed due to %s" + .format(offsetCommitRequest.correlationId, offsetCommitRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(errorCode))) } } - } - private def producerRequestFromOffsetCommit(offsetCommitRequest: OffsetCommitRequest) = { - val msgs = offsetCommitRequest.filterLargeMetadata(config.offsetMetadataMaxSize).map { - case (topicAndPartition, offset) => - new Message( - bytes = OffsetManager.offsetCommitValue(offset), - key = OffsetManager.offsetCommitKey(offsetCommitRequest.groupId, topicAndPartition.topic, topicAndPartition.partition) - ) - }.toSeq - - val producerData = mutable.Map( - TopicAndPartition(OffsetManager.OffsetsTopicName, offsetManager.partitionFor(offsetCommitRequest.groupId)) -> - new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, msgs:_*) - ) + val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } - val request = ProducerRequest( - correlationId = offsetCommitRequest.correlationId, - clientId = offsetCommitRequest.clientId, - requiredAcks = config.offsetCommitRequiredAcks, - ackTimeoutMs = config.offsetCommitTimeoutMs, - data = producerData) - trace("Created producer request %s for offset commit request %s.".format(request, offsetCommitRequest)) - request + // call offset manager to store offsets + offsetManager.storeOffsets( + offsetCommitRequest.groupId, + offsetCommitRequest.consumerId, + offsetCommitRequest.groupGenerationId, + offsetCommitRequest.requestInfo, + sendResponseCallback) } /** diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 43eb2a3..c4d6747 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -35,11 +35,13 @@ import scala.collection._ import java.io.PrintStream import java.util.concurrent.atomic.AtomicBoolean import java.nio.ByteBuffer -import java.util.Properties +import java.util.{Collections, Properties} import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge import org.I0Itec.zkclient.ZkClient +import kafka.api.{ProducerResponse, ProducerResponseStatus} +import kafka.network.{BoundedByteBufferSend, RequestChannel} /** @@ -192,13 +194,87 @@ class OffsetManager(val config: OffsetManagerConfig, offsetsCache.put(key, offsetAndMetadata) } - def putOffsets(group: String, offsets: Map[TopicAndPartition, OffsetAndMetadata]) { - // this method is called _after_ the offsets have been durably appended to the commit log, so there is no need to - // check for current leadership as we do for the offset fetch - trace("Putting offsets %s for group %s in offsets partition %d.".format(offsets, group, partitionFor(group))) - offsets.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(group, topicAndPartition), offsetAndMetadata) + /** + * Store offsets by appending it to the replicated log and then inserting to cache + */ + def storeOffsets(groupName: String, + consumerId: String, + generationId: Int, + offsetMetadata: Map[TopicAndPartition, OffsetAndMetadata], + callbackOnComplete: Map[TopicAndPartition, Short] => Unit) { + // TODO: generation id and consumer id is needed by coordinator to do consumer checking + + // first filter out partitions with offset metadata size exceeding limit + // TODO: in the future we may want to only support atomic commit and hence fail the whole commit when this happens + var commitStatus = offsetMetadata.mapValues { offsetAndMetadata => + if (offsetAndMetadata.metadata != null && offsetAndMetadata.metadata.lengh > maxMetadataSize) + ErrorMapping.OffsetMetadataTooLargeCode + else + ErrorMapping.NoError + } + + val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => + commitStatus.get(TopicAndPartition).get == ErrorMapping.NoError + } + + // construct the message set to append + val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + new Message( + key = OffsetManager.offsetCommitKey(groupName, topicAndPartition.topic, topicAndPartition.partition), + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) + ) + }.toSeq + + val offsetTopicPartition = TopicAndPartition(OffsetsTopicName, partitionFor(groupName)) + + val messageSet = Collections.singletonMap(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 iff the append status has no error + val status = responseStatus.get(offsetTopicPartition).get + + if (status.error == ErrorMapping.NoError) { + filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => + putOffset(GroupTopicPartition(group, topicAndPartition), offsetAndMetadata) + } + } else { + debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" + .format(filteredOffsetMetadata, groupName, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) + + // update the commit status error code with the corresponding log append error code + val commitErrorCode = + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else + status.error + + commitStatus = commitStatus.mapValues { case errorCode => + if (errorCode == ErrorMapping.NoError) + commitErrorCode + else + errorCode + } + } + + callbackOnComplete(commitStatus) } + + // call replica manager to append the offset messages + replicaManager.appendMessages( + config.offsetCommitTimeoutMs.toLong, + config.offsetCommitRequiredAcks, + messageSet, + putCacheCallback) } /** diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 8b55063..e77b9d2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -77,8 +77,9 @@ class ReplicaManager(config: KafkaConfig, this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger - var producerRequestPurgatory: ProducerRequestPurgatory = null - var fetchRequestPurgatory: FetchRequestPurgatory = null + val producerRequestPurgatory = new RequestPurgatory[DelayedProduce](brokerId, config.producerPurgatoryPurgeIntervalRequests) + val fetchRequestPurgatory = new RequestPurgatory[DelayedFetch](brokerId, config.fetchPurgatoryPurgeIntervalRequests) + newGauge( "LeaderCount", @@ -113,37 +114,26 @@ class ReplicaManager(config: KafkaConfig, } /** - * Initialize the replica manager with the request purgatory - * - * TODO: will be removed in 0.9 where we refactor server structure - */ - - def initWithRequestPurgatory(producerRequestPurgatory: ProducerRequestPurgatory, fetchRequestPurgatory: FetchRequestPurgatory) { - this.producerRequestPurgatory = producerRequestPurgatory - this.fetchRequestPurgatory = fetchRequestPurgatory - } - - /** * Unblock some delayed produce requests with the request key */ def unblockDelayedProduceRequests(key: DelayedRequestKey) { - val satisfied = producerRequestPurgatory.update(key) + val satisfied = producerRequestPurgatory.getCompleted(key) debug("Request key %s unblocked %d producer requests." .format(key.keyLabel, satisfied.size)) - // send any newly unblocked responses - satisfied.foreach(producerRequestPurgatory.respond(_)) + // complete the produce operation + satisfied.foreach(_.completeProduce()) } /** * Unblock some delayed fetch requests with the request key */ def unblockDelayedFetchRequests(key: DelayedRequestKey) { - val satisfied = fetchRequestPurgatory.update(key) + val satisfied = fetchRequestPurgatory.getCompleted(key) debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, satisfied.size)) - // send any newly unblocked responses - satisfied.foreach(fetchRequestPurgatory.respond(_)) + // complete the fetch operation + satisfied.foreach(_.completeFetch()) } def startup() { @@ -285,10 +275,8 @@ class ReplicaManager(config: KafkaConfig, val producerRequestKeys = messagesPerPartition.keys.map(TopicPartitionRequestKey(_)).toSeq val completedByMe = producerRequestPurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) - if (completedByMe) { - val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) - callbackOnComplete(produceResponseStatus) - } + if (completedByMe) + delayedProduce.completeProduce() } } @@ -361,7 +349,7 @@ class ReplicaManager(config: KafkaConfig, // if the fetch comes from the follower, // update its corresponding log end offset if(Request.isValidBrokerId(fetchRequest.replicaId)) - recordFollowerLogEndOffsets(replicaId, dataRead.mapValues(_.offset)) + updateFollowerLEOs(replicaId, dataRead.mapValues(_.offset)) // check if this fetch request can be satisfied right away val bytesReadable = logReadResults.values.map(_.info.messageSet.sizeInBytes).sum @@ -387,12 +375,8 @@ class ReplicaManager(config: KafkaConfig, // add the fetch request for watch if it's not satisfied, otherwise send the response back val completedByMe = fetchRequestPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) - if (completedByMe) { - // fetch again to get whatever is available - val fetchPartitionData = readFromLocalLog(fetchOnlyLeader, fetchOnlyCommitted, fetchInfo) - .mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - callbackOnComplete(fetchPartitionData) - } + if (completedByMe) + delayedFetch.completeFetch() } } @@ -452,18 +436,6 @@ class ReplicaManager(config: KafkaConfig, } } - private def recordFollowerLogEndOffsets(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { - debug("Record follower log end offsets: %s ".format(offsets)) - offsets.foreach { - case (topicAndPartition, offset) => - updateReplicaLEOAndPartitionHW(topicAndPartition.topic, topicAndPartition.partition, replicaId, offset) - - // for producer requests with ack > 1, we need to check - // if they can be unblocked after some follower's log end offsets have moved - unblockDelayedProduceRequests(new TopicPartitionRequestKey(topicAndPartition)) - } - } - def maybeUpdateMetadataCache(updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { replicaStateChangeLock synchronized { if(updateMetadataRequest.controllerEpoch < controllerEpoch) { @@ -714,15 +686,29 @@ class ReplicaManager(config: KafkaConfig, allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages)) } - def updateReplicaLEOAndPartitionHW(topic: String, partitionId: Int, replicaId: Int, offset: LogOffsetMetadata) = { - getPartition(topic, partitionId) match { + private def updateFollowerLEOs(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { + debug("Recording follower broker %d log end offsets: %s ".format(replicaId, offsets)) + offsets.foreach { + case (topicAndPartition, offset) => + updateReplicaLEO(topicAndPartition, replicaId, offset) + + // for producer requests with ack > 1, we need to check + // if they can be unblocked after some follower's log end offsets have moved + unblockDelayedProduceRequests(new TopicPartitionRequestKey(topicAndPartition)) + } + } + + private def updateReplicaLEO(topicAndPartition: TopicAndPartition, replicaId: Int, offset: LogOffsetMetadata) = { + getPartition(topicAndPartition.topic, topicAndPartition.partition) match { case Some(partition) => partition.getReplica(replicaId) match { case Some(replica) => replica.logEndOffset = offset - // check if we need to update HW and expand Isr + + // check if we need to update HW and expand Isr after some of its replica's LEOs have changed partition.updateLeaderHWAndMaybeExpandIsr(replicaId) - debug("Recorded follower %d position %d for partition [%s,%d].".format(replicaId, offset.messageOffset, topic, partitionId)) + + debug("Recorded replica %d LEO position %d for partition %s.".format(replicaId, offset.messageOffset, topicAndPartition)) case None => throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + " is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId, @@ -730,7 +716,7 @@ class ReplicaManager(config: KafkaConfig, } case None => - warn("While recording the follower position, the partition [%s,%d] hasn't been created, skip updating leader HW".format(topic, partitionId)) + warn("While recording the replica LEO, the partition %s hasn't been created.".format(topicAndPartition)) } } diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index ebfcd71..c09e55b 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -62,7 +62,7 @@ abstract class DelayedRequest(delayMs: Long, onComplete: Any => Unit) extends De * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. * */ -abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: Int = 1000) +class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: Int = 1000) extends Logging with KafkaMetricsGroup { /* a list of requests watching each key */ diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 09ed8f5..5ebd585 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -95,7 +95,6 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.reset(replicaManager) EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.expect(replicaManager.initWithRequestPurgatory(EasyMock.anyObject(), EasyMock.anyObject())) EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ val fetchInfo = log.read(0, fetchSize, Some(hw)) val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) @@ -184,10 +183,9 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.reset(replicaManager) EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.updateReplicaLEOAndPartitionHW(topic, partitionId, followerReplicaId, new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, followerLEO))) + EasyMock.expect(replicaManager.updateReplicaLEO(TopicAndPartition(topic, partitionId), followerReplicaId, new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, followerLEO))) EasyMock.expect(replicaManager.getReplica(topic, partitionId, followerReplicaId)).andReturn(partition.inSyncReplicas.find(_.brokerId == configs(1).brokerId)) EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.expect(replicaManager.initWithRequestPurgatory(EasyMock.anyObject(), EasyMock.anyObject())) EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ val fetchInfo = log.read(followerLEO, Integer.MAX_VALUE, None) val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) -- 1.7.12.4 From 8854b288bd38e558cf889d7c46c5c8fe82c07eb9 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 18 Aug 2014 15:08:43 -0700 Subject: [PATCH 10/33] fix 1 --- core/src/main/scala/kafka/server/DelayedFetch.scala | 7 ++++--- core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 5cff718..0cba36c 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -37,8 +37,8 @@ import scala.collection.immutable.Map case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInfo: PartitionFetchInfo) { - override def toString = "[startOffset: " + startOffsetMetadata + ", " + - "fetchSize: " + fetchSize + "]" + override def toString = "[startOffsetMetadata: " + startOffsetMetadata + ", " + + "fetchInfo: " + fetchInfo + "]" } case class FetchMetadata(fetchMinBytes: Int, @@ -61,7 +61,8 @@ class DelayedFetch(delayMs: Long, override def tryComplete() : Boolean = { var accumulatedSize = 0 fetchMetadata.fetchPartitionStatus.foreach { - case (topicAndPartition, fetchOffset) => + case (topicAndPartition, fetchStatus) => + val fetchOffset = fetchStatus.startOffsetMetadata try { if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) { val replica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e77b9d2..ff1f4ad 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -59,7 +59,7 @@ case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Op } } -class ReplicaManager(config: KafkaConfig, +class ReplicaManager(val config: KafkaConfig, time: Time, val zkClient: ZkClient, scheduler: Scheduler, -- 1.7.12.4 From 72a5088f6ab84f48700feef57e0453cc27b5041e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 18 Aug 2014 17:35:21 -0700 Subject: [PATCH 11/33] fix 2 --- .../src/main/scala/kafka/server/DelayedFetch.scala | 5 +- .../main/scala/kafka/server/DelayedProduce.scala | 2 +- .../scala/kafka/server/FetchRequestPurgatory.scala | 69 ---------------------- core/src/main/scala/kafka/server/KafkaApis.scala | 5 +- .../kafka/server/ProducerRequestPurgatory.scala | 69 ---------------------- .../main/scala/kafka/server/ReplicaManager.scala | 18 +++--- 6 files changed, 14 insertions(+), 154 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/FetchRequestPurgatory.scala delete mode 100644 core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 0cba36c..19dbcc4 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -47,7 +47,8 @@ case class FetchMetadata(fetchMinBytes: Int, fetchPartitionStatus: Map[TopicAndPartition, FetchPartitionStatus]) { override def toString = "[minBytes: " + fetchMinBytes + ", " + - "committedOnly: " + fetchOnlyCommitted + ", " + "onlyLeader:" + fetchOnlyLeader + ", " + "onlyCommitted: " + fetchOnlyCommitted + ", " "partitionStatus: " + fetchPartitionStatus + "]" } @@ -55,7 +56,7 @@ case class FetchMetadata(fetchMinBytes: Int, class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, - onComplete: Map[TopicAndPartition, PartitionDataAndOffset] => Unit) + onComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) extends DelayedRequest(delayMs, onComplete) { override def tryComplete() : Boolean = { diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index bbb176c..ee66abe 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -51,7 +51,7 @@ class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, onComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) - extends DelayedRequest(delayMs) with Logging { + extends DelayedRequest(delayMs, onComplete) with Logging { // first update the acks pending variable according to the error code produceMetadata.produceStatus foreach { case (topicAndPartition, status) => diff --git a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala b/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala deleted file mode 100644 index ed13188..0000000 --- a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala +++ /dev/null @@ -1,69 +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 kafka.metrics.KafkaMetricsGroup -import kafka.network.RequestChannel -import kafka.api.FetchResponseSend - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed fetch requests - */ -class FetchRequestPurgatory(replicaManager: ReplicaManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedFetch](replicaManager.config.brokerId, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) { - this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup { - private val metricPrefix = if (forFollower) "Follower" else "Consumer" - - val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) - private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) - - private def recordDelayedFetchExpired(forFollower: Boolean) { - val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics - else aggregateNonFollowerFetchRequestMetrics - - metrics.expiredRequestMeter.mark() - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedFetch: DelayedFetch): Boolean = delayedFetch.isSatisfied(replicaManager) - - /** - * When a delayed fetch request expires just answer it with whatever data is present - */ - def expire(delayedFetch: DelayedFetch) { - debug("Expiring fetch request %s.".format(delayedFetch.fetch)) - val fromFollower = delayedFetch.fetch.isFromFollower - recordDelayedFetchExpired(fromFollower) - respond(delayedFetch) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedFetch: DelayedFetch) { - val response = delayedFetch.respond(replicaManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedFetch.request, new FetchResponseSend(response))) - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 98b4e35..088da6a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -20,7 +20,6 @@ package kafka.server import kafka.api._ import kafka.common._ import kafka.log._ -import kafka.message._ import kafka.network._ import kafka.admin.AdminUtils import kafka.network.RequestChannel.Response @@ -123,8 +122,8 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] // the callback for sending the response - def sendResponseCallback(responseStatus: Map[TopicAndPartition, Short]) { - responseStatus.foreach { case (topicAndPartition, errorCode) => + def sendResponseCallback(commitStatus: Map[TopicAndPartition, Short]) { + commitStatus.foreach { case (topicAndPartition, errorCode) => // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here if (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.UnknownCode) { diff --git a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala b/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala deleted file mode 100644 index d4a7d4a..0000000 --- a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala +++ /dev/null @@ -1,69 +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 kafka.metrics.KafkaMetricsGroup -import kafka.utils.Pool -import kafka.network.{BoundedByteBufferSend, RequestChannel} - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed producer requests - */ -class ProducerRequestPurgatory(replicaManager: ReplicaManager, offsetManager: OffsetManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) { - this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup { - val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val producerRequestMetricsForKey = { - val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") - new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory)) - } - - private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics - - private def recordDelayedProducerKeyExpired(key: DelayedRequestKey) { - val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) - List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedProduce: DelayedProduce) = delayedProduce.isSatisfied(replicaManager) - - /** - * When a delayed produce request expires answer it with possible time out error codes - */ - def expire(delayedProduce: DelayedProduce) { - debug("Expiring produce request %s.".format(delayedProduce.produce)) - for ((topicPartition, responseStatus) <- delayedProduce.partitionStatus if responseStatus.acksPending) - recordDelayedProducerKeyExpired(new TopicPartitionRequestKey(topicPartition)) - respond(delayedProduce) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedProduce: DelayedProduce) { - val response = delayedProduce.respond(offsetManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedProduce.request, new BoundedByteBufferSend(response))) - } -} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index ff1f4ad..7bf374d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -43,8 +43,6 @@ object ReplicaManager { val HighWatermarkFilename = "replication-offset-checkpoint" } -case class PartitionDataAndOffset(data: FetchResponsePartitionData, offset: LogOffsetMetadata) - case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { def errorCode = error match { case None => ErrorMapping.NoError @@ -283,7 +281,7 @@ class ReplicaManager(val config: KafkaConfig, /** * Append the messages to the local replica logs */ - private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]) = { + private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]): immutable.Map[TopicAndPartition, LogAppendResult] = { trace("Append [%s] to local log ".format(messagesPerPartition)) messagesPerPartition.map { case (topicAndPartition, messages) => try { @@ -385,7 +383,7 @@ class ReplicaManager(val config: KafkaConfig, */ def readFromLocalLog(readOnlyIfLeader: Boolean, readOnlyCommitted: Boolean, - readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]) = { + readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): immutable.Map[TopicAndPartition, LogReadResult] = { readPartitionInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => val partitionDataAndOffsetInfo = @@ -407,7 +405,7 @@ class ReplicaManager(val config: KafkaConfig, // read on log val logReadInfo = localReplica.log match { case Some(log) => - log.read(offset, maxSize, maxOffsetOpt) + log.read(offset, fetchSize, maxOffsetOpt) case None => error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) @@ -416,21 +414,21 @@ class ReplicaManager(val config: KafkaConfig, BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) - LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, ErrorMapping.NoError) + LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, None) } catch { // NOTE: Failed fetch requests metric is not incremented for known exceptions since it // is supposed to indicate un-expected failure of a broker in handling a fetch request case utpe: UnknownTopicOrPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, utpe) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(utpe)) case nle: NotLeaderForPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, nle) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(nle)) case rnae: ReplicaNotAvailableException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, rnae) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(rnae)) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() error("Error processing fetch operation on partition [%s,%d] offset %d".format(topic, partition, offset)) - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, t) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(e)) } (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) } -- 1.7.12.4 From dcddcef1ab84110a1111a1b2ffbedee3845d4656 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 18 Aug 2014 17:44:09 -0700 Subject: [PATCH 12/33] fix 3 --- core/src/main/scala/kafka/server/KafkaApis.scala | 7 ++----- core/src/main/scala/kafka/server/OffsetManager.scala | 4 ++-- core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 088da6a..5164618 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -122,7 +122,7 @@ class KafkaApis(val requestChannel: RequestChannel, val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] // the callback for sending the response - def sendResponseCallback(commitStatus: Map[TopicAndPartition, Short]) { + def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { commitStatus.foreach { case (topicAndPartition, errorCode) => // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here @@ -201,7 +201,7 @@ class KafkaApis(val requestChannel: RequestChannel, val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] // the callback for sending the response - def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + def sendResponseCallback(responsePartitionData: immutable.Map[TopicAndPartition, FetchResponsePartitionData]) { responsePartitionData.foreach { case (topicAndPartition, data) => // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here @@ -416,9 +416,6 @@ class KafkaApis(val requestChannel: RequestChannel, } def close() { - debug("Shutting down.") - fetchRequestPurgatory.shutdown() - producerRequestPurgatory.shutdown() debug("Shut down complete.") } } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index c4d6747..1bbfd14 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -200,8 +200,8 @@ class OffsetManager(val config: OffsetManagerConfig, def storeOffsets(groupName: String, consumerId: String, generationId: Int, - offsetMetadata: Map[TopicAndPartition, OffsetAndMetadata], - callbackOnComplete: Map[TopicAndPartition, Short] => Unit) { + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + callbackOnComplete: immutable.Map[TopicAndPartition, Short] => Unit) { // TODO: generation id and consumer id is needed by coordinator to do consumer checking // first filter out partitions with offset metadata size exceeding limit diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 7bf374d..20fbcad 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -336,7 +336,7 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchMinBytes: Int, fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], - callbackOnComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + callbackOnComplete: immutable.Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { val fetchOnlyLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(fetchRequest.replicaId) -- 1.7.12.4 From eb9a30ce15a76d2a8756ffe3eefae114e732538b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 18 Aug 2014 19:21:11 -0700 Subject: [PATCH 13/33] fix 4 --- core/src/main/scala/kafka/log/Log.scala | 13 ++++++ .../main/scala/kafka/server/OffsetManager.scala | 16 ++++---- .../main/scala/kafka/server/ReplicaManager.scala | 48 ++++++++-------------- .../main/scala/kafka/server/RequestPurgatory.scala | 12 +++--- core/src/main/scala/kafka/utils/DelayedItem.scala | 2 +- 5 files changed, 48 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index b381bb1..567bf23 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -46,6 +46,19 @@ object LogAppendInfo { val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, -1, -1, false) } +case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} + +case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} /** * An append-only log for storing messages. diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 1bbfd14..14e0cc6 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -40,8 +40,10 @@ import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge import org.I0Itec.zkclient.ZkClient -import kafka.api.{ProducerResponse, ProducerResponseStatus} -import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.api.ProducerResponseStatus + +import collection.JavaConversions._ + /** @@ -207,14 +209,14 @@ class OffsetManager(val config: OffsetManagerConfig, // first filter out partitions with offset metadata size exceeding limit // TODO: in the future we may want to only support atomic commit and hence fail the whole commit when this happens var commitStatus = offsetMetadata.mapValues { offsetAndMetadata => - if (offsetAndMetadata.metadata != null && offsetAndMetadata.metadata.lengh > maxMetadataSize) + if (offsetAndMetadata.metadata != null && offsetAndMetadata.metadata.length() > config.maxMetadataSize) ErrorMapping.OffsetMetadataTooLargeCode else ErrorMapping.NoError } val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - commitStatus.get(TopicAndPartition).get == ErrorMapping.NoError + commitStatus.get(topicAndPartition).get == ErrorMapping.NoError } // construct the message set to append @@ -225,10 +227,10 @@ class OffsetManager(val config: OffsetManagerConfig, ) }.toSeq - val offsetTopicPartition = TopicAndPartition(OffsetsTopicName, partitionFor(groupName)) + val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupName)) val messageSet = Collections.singletonMap(offsetTopicPartition, - new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)).toMap // set the callback function to insert offsets into cache after log append completed def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { @@ -243,7 +245,7 @@ class OffsetManager(val config: OffsetManagerConfig, if (status.error == ErrorMapping.NoError) { filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(group, topicAndPartition), offsetAndMetadata) + putOffset(GroupTopicPartition(groupName, topicAndPartition), offsetAndMetadata) } } else { debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 20fbcad..c17c511 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -20,7 +20,7 @@ import kafka.api._ import kafka.common._ import kafka.utils._ import kafka.cluster.{Broker, Partition, Replica} -import kafka.log.{LogAppendInfo, LogManager} +import kafka.log.{LogReadResult, LogAppendResult, LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.common.TopicAndPartition @@ -43,20 +43,6 @@ object ReplicaManager { val HighWatermarkFilename = "replication-offset-checkpoint" } -case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { - def errorCode = error match { - case None => ErrorMapping.NoError - case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) - } -} - -case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Option[Throwable] = None) { - def errorCode = error match { - case None => ErrorMapping.NoError - case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) - } -} - class ReplicaManager(val config: KafkaConfig, time: Time, val zkClient: ZkClient, @@ -75,8 +61,8 @@ class ReplicaManager(val config: KafkaConfig, this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger - val producerRequestPurgatory = new RequestPurgatory[DelayedProduce](brokerId, config.producerPurgatoryPurgeIntervalRequests) - val fetchRequestPurgatory = new RequestPurgatory[DelayedFetch](brokerId, config.fetchPurgatoryPurgeIntervalRequests) + val producerRequestPurgatory = new RequestPurgatory[DelayedProduce](config.brokerId, config.producerPurgatoryPurgeIntervalRequests) + val fetchRequestPurgatory = new RequestPurgatory[DelayedFetch](config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) newGauge( @@ -260,9 +246,9 @@ class ReplicaManager(val config: KafkaConfig, // if required acks = 0 we can trigger complete immediately val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) callbackOnComplete(produceResponseStatus) - } else if (produceRequest.requiredAcks == 1 || + } else if (requiredAcks == 1 || messagesPerPartition.size <= 0 || - localProduceResults.values.count(_.error.isDefined) == produceRequest.numPartitions) { + localProduceResults.values.count(_.error.isDefined) == messagesPerPartition.size) { // if required acks = 1 or all partition appends have failed we can trigger complete immediately val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) callbackOnComplete(produceResponseStatus) @@ -270,7 +256,7 @@ class ReplicaManager(val config: KafkaConfig, // create delayed produce operation and try to watch it in the purgatory val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, callbackOnComplete) - val producerRequestKeys = messagesPerPartition.keys.map(TopicPartitionRequestKey(_)).toSeq + val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionRequestKey(_)).toSeq val completedByMe = producerRequestPurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) if (completedByMe) @@ -316,14 +302,14 @@ class ReplicaManager(val config: KafkaConfig, Runtime.getRuntime.halt(1) (topicAndPartition, null) case utpe: UnknownTopicOrPartitionException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, utpe)) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) case nle: NotLeaderForPartitionException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, nle)) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() error("Error processing append operation on partition %s".format(topicAndPartition), e) - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, e)) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e))) } } } @@ -339,15 +325,15 @@ class ReplicaManager(val config: KafkaConfig, callbackOnComplete: immutable.Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { val fetchOnlyLeader: Boolean = replicaId != Request.DebuggingConsumerId - val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(fetchRequest.replicaId) + val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) // read from local logs val logReadResults = readFromLocalLog(fetchOnlyLeader, fetchOnlyCommitted, fetchInfo) // if the fetch comes from the follower, // update its corresponding log end offset - if(Request.isValidBrokerId(fetchRequest.replicaId)) - updateFollowerLEOs(replicaId, dataRead.mapValues(_.offset)) + if(Request.isValidBrokerId(replicaId)) + updateFollowerLEOs(replicaId, logReadResults.mapValues(_.info.fetchOffset)) // check if this fetch request can be satisfied right away val bytesReadable = logReadResults.values.map(_.info.messageSet.sizeInBytes).sum @@ -364,9 +350,11 @@ class ReplicaManager(val config: KafkaConfig, callbackOnComplete(fetchPartitionData) } else { // construct the fetch results from the read results - val fetchPartitionStatus = logReadResults.mapValues(result => FetchPartitionStatus(result.info.fetchOffset, result.readSize)) + val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => + (topicAndPartition, FetchPartitionStatus(result.info.fetchOffset, fetchInfo.get(topicAndPartition).get)) + } val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyLeader, fetchOnlyCommitted, fetchPartitionStatus) - val delayedFetch = new DelayedFetch(time, fetchMetadata, this, callbackOnComplete) + val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, callbackOnComplete) // create a list of (topic, partition) pairs to use as keys for this delayed request val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionRequestKey(_)).toSeq @@ -709,8 +697,8 @@ class ReplicaManager(val config: KafkaConfig, debug("Recorded replica %d LEO position %d for partition %s.".format(replicaId, offset.messageOffset, topicAndPartition)) case None => throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + - " is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId, - offset.messageOffset, partition.assignedReplicas().map(_.brokerId).mkString(","), topic, partitionId)) + " is not recognized to be one of the assigned replicas %s for partition %s").format(localBrokerId, replicaId, + offset.messageOffset, partition.assignedReplicas().map(_.brokerId).mkString(","), topicAndPartition)) } case None => diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index c09e55b..6c99cbb 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -81,11 +81,11 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In newGauge( "NumDelayedOperations", new Gauge[Int] { - def value = expirationReaper.numOperations + def value = expirationReaper.enqueued } ) - expirationThread.start() + expirationReaper.start() /** * Check if the operation can be completed, if not watch it based on the given watch keys @@ -99,7 +99,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In * @param watchKeys keys for bookkeeping the operation * @return true iff the delayed operations can be completed */ - def tryCompleteElseWatch(operation: DelayedRequest, watchKeys: Seq[Any]): Boolean = { + def tryCompleteElseWatch(operation: T, watchKeys: Seq[Any]): Boolean = { for(key <- watchKeys) { // if the operation is already completed, stopping adding it to // any further lists and return false @@ -144,7 +144,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In * Since an operation may still be in the watch lists even when it has been completed, this number * may be larger than the number of real operations watched */ - protected def size() = watchersForKey.values.map(_.numRequests).sum + expirationReaper.numOperations + protected def size() = watchersForKey.values.map(_.watched).sum + expirationReaper.enqueued /** * Shutdown the expire reaper thread @@ -159,6 +159,8 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In private class Watchers { private val requests = new util.LinkedList[T] + def watched = requests.size() + // potentially add the element to watch if it is not satisfied yet def checkAndMaybeAdd(t: T): Boolean = { synchronized { @@ -224,7 +226,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In /* * Return the number of delayed operations kept by the reaper */ - def numOperations = delayed.size() + def enqueued = delayed.size() /* * Add a operation to be expired diff --git a/core/src/main/scala/kafka/utils/DelayedItem.scala b/core/src/main/scala/kafka/utils/DelayedItem.scala index 3d7df84..a4e0dab 100644 --- a/core/src/main/scala/kafka/utils/DelayedItem.scala +++ b/core/src/main/scala/kafka/utils/DelayedItem.scala @@ -41,7 +41,7 @@ class DelayedItem(delay: Long, unit: TimeUnit) extends Delayed with Logging { } def compareTo(d: Delayed): Int = { - val delayed = d.asInstanceOf[DelayedItem[T]] + val delayed = d.asInstanceOf[DelayedItem] val myEnd = createdMs + delayMs val yourEnd = delayed.createdMs + delayed.delayMs -- 1.7.12.4 From daf82177544d67d6a0784f434afb89d42ece7871 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 19 Aug 2014 16:31:19 -0700 Subject: [PATCH 14/33] compilation passed --- core/src/main/scala/kafka/api/FetchResponse.scala | 12 ++++++------ core/src/main/scala/kafka/log/Log.scala | 3 ++- .../src/main/scala/kafka/network/BoundedByteBufferSend.scala | 4 ++-- core/src/main/scala/kafka/server/DelayedFetch.scala | 8 ++++---- core/src/main/scala/kafka/server/DelayedProduce.scala | 8 ++++---- core/src/main/scala/kafka/server/KafkaApis.scala | 4 ++-- core/src/main/scala/kafka/server/ReplicaManager.scala | 6 +++--- core/src/main/scala/kafka/server/RequestPurgatory.scala | 2 +- 8 files changed, 24 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 8d085a1..75aaf57 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -25,6 +25,8 @@ import kafka.message.{MessageSet, ByteBufferMessageSet} import kafka.network.{MultiSend, Send} import kafka.api.ApiUtils._ +import scala.collection._ + object FetchResponsePartitionData { def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { val error = buffer.getShort @@ -150,10 +152,8 @@ object FetchResponse { } } - -case class FetchResponse(correlationId: Int, - data: Map[TopicAndPartition, FetchResponsePartitionData]) - extends RequestOrResponse() { +case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData]) + extends RequestOrResponse() { /** * Partitions the data into a map of maps (one for each topic). @@ -171,8 +171,8 @@ case class FetchResponse(correlationId: Int, /* * FetchResponse uses [sendfile](http://man7.org/linux/man-pages/man2/sendfile.2.html) - * api for data transfer, so `writeTo` aren't actually being used. - * It is implemented as an empty function to comform to `RequestOrResponse.writeTo` + * api for data transfer through the FetchResponseSend, so `writeTo` aren't actually being used. + * It is implemented as an empty function to conform to `RequestOrResponse.writeTo` * abstract method signature. */ def writeTo(buffer: ByteBuffer): Unit = throw new UnsupportedOperationException diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 567bf23..6008135 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -40,12 +40,13 @@ import com.yammer.metrics.core.Gauge * @param codec The codec used in the message set * @param offsetsMonotonic Are the offsets in this message set monotonically increasing */ -case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) object LogAppendInfo { val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, -1, -1, false) } +case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) + case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { def errorCode = error match { case None => ErrorMapping.NoError diff --git a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala b/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala index a624359..55ecac2 100644 --- a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala +++ b/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala @@ -25,7 +25,7 @@ import kafka.api.RequestOrResponse @nonthreadsafe private[kafka] class BoundedByteBufferSend(val buffer: ByteBuffer) extends Send { - private var sizeBuffer = ByteBuffer.allocate(4) + private val sizeBuffer = ByteBuffer.allocate(4) // Avoid possibility of overflow for 2GB-4 byte buffer if(buffer.remaining > Int.MaxValue - sizeBuffer.limit) @@ -53,7 +53,7 @@ private[kafka] class BoundedByteBufferSend(val buffer: ByteBuffer) extends Send def writeTo(channel: GatheringByteChannel): Int = { expectIncomplete() - var written = channel.write(Array(sizeBuffer, buffer)) + val written = channel.write(Array(sizeBuffer, buffer)) // if we are done, mark it off if(!buffer.hasRemaining) complete = true diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 19dbcc4..6e17027 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -20,7 +20,7 @@ package kafka.server import kafka.api.{FetchResponsePartitionData, PartitionFetchInfo, FetchResponse, FetchRequest} import kafka.common.{UnknownTopicOrPartitionException, NotLeaderForPartitionException, TopicAndPartition} -import scala.collection.immutable.Map +import scala.collection._ /** * A delayed fetch request, which is satisfied (or more @@ -56,8 +56,8 @@ case class FetchMetadata(fetchMinBytes: Int, class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, - onComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) - extends DelayedRequest(delayMs, onComplete) { + callbackOnComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) + extends DelayedRequest(delayMs) { override def tryComplete() : Boolean = { var accumulatedSize = 0 @@ -116,6 +116,6 @@ class DelayedFetch(delayMs: Long, val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - onComplete(fetchPartitionData) + callbackOnComplete(fetchPartitionData) } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index ee66abe..846fc97 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -23,7 +23,7 @@ import kafka.common.TopicAndPartition import kafka.utils.Logging import scala.Some -import scala.collection.immutable.Map +import scala.collection._ /** A delayed produce request, which is satisfied (or more * accurately, unblocked) -- if for every partition it produce to: @@ -50,8 +50,8 @@ case class ProduceMetadata(produceRequiredAcks: Short, class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, - onComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) - extends DelayedRequest(delayMs, onComplete) with Logging { + callbackOnComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) + extends DelayedRequest(delayMs) { // first update the acks pending variable according to the error code produceMetadata.produceStatus foreach { case (topicAndPartition, status) => @@ -107,7 +107,7 @@ class DelayedProduce(delayMs: Long, def completeProduce() { // return the current response status val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) - onComplete(responseStatus) + callbackOnComplete(responseStatus) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5164618..b3a5d30 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -201,7 +201,7 @@ class KafkaApis(val requestChannel: RequestChannel, val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] // the callback for sending the response - def sendResponseCallback(responsePartitionData: immutable.Map[TopicAndPartition, FetchResponsePartitionData]) { + def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { responsePartitionData.foreach { case (topicAndPartition, data) => // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here @@ -213,7 +213,7 @@ class KafkaApis(val requestChannel: RequestChannel, } val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) } // call the replica manager to append messages to the replicas diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index c17c511..4ccd80d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -267,7 +267,7 @@ class ReplicaManager(val config: KafkaConfig, /** * Append the messages to the local replica logs */ - private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]): immutable.Map[TopicAndPartition, LogAppendResult] = { + private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]): Map[TopicAndPartition, LogAppendResult] = { trace("Append [%s] to local log ".format(messagesPerPartition)) messagesPerPartition.map { case (topicAndPartition, messages) => try { @@ -322,7 +322,7 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchMinBytes: Int, fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], - callbackOnComplete: immutable.Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + callbackOnComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { val fetchOnlyLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) @@ -371,7 +371,7 @@ class ReplicaManager(val config: KafkaConfig, */ def readFromLocalLog(readOnlyIfLeader: Boolean, readOnlyCommitted: Boolean, - readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): immutable.Map[TopicAndPartition, LogReadResult] = { + readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { readPartitionInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => val partitionDataAndOffsetInfo = diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 6c99cbb..ee257de 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -34,7 +34,7 @@ import com.yammer.metrics.core.Gauge * message append operation could be waiting for specified number of acks; or a delayed * message fetch operation could be waiting for a given number of bytes to accumulate. */ -abstract class DelayedRequest(delayMs: Long, onComplete: Any => Unit) extends DelayedItem(delayMs) { +abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { val completed = new AtomicBoolean(false) /* -- 1.7.12.4 From a9a8d16d8344a64808f1d3f7f74dbf1a31a13c16 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 20 Aug 2014 13:53:47 -0700 Subject: [PATCH 15/33] comments --- .../src/main/scala/kafka/api/ProducerRequest.scala | 5 - core/src/main/scala/kafka/cluster/Partition.scala | 39 +++++++- .../src/main/scala/kafka/server/DelayedFetch.scala | 54 ++++++----- .../main/scala/kafka/server/DelayedProduce.scala | 43 ++++++--- core/src/main/scala/kafka/server/KafkaApis.scala | 13 +-- .../main/scala/kafka/server/OffsetManager.scala | 8 +- .../main/scala/kafka/server/ReplicaManager.scala | 96 ++++++++---------- .../main/scala/kafka/server/RequestPurgatory.scala | 107 +++++++++++---------- .../scala/unit/kafka/server/SimpleFetchTest.scala | 2 +- 9 files changed, 206 insertions(+), 161 deletions(-) diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index b2366e7..b062406 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -152,10 +152,5 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, producerRequest.append("; TopicAndPartition: " + topicPartitionMessageSizeMap.mkString(",")) producerRequest.toString() } - - - def emptyData(){ - data.clear() - } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index e88ecf2..a4a4c2f 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -230,7 +230,31 @@ class Partition(val topic: String, } } - def updateLeaderHWAndMaybeExpandIsr(replicaId: Int) { + /** + * Update the log end offset of a replica of the partition + */ + def updateReplicaLEO(replicaId: Int, offset: LogOffsetMetadata) = { + getReplica(replicaId) match { + case Some(replica) => + replica.logEndOffset = offset + + // check if we need to expand Isr to include this replica after its LEO has advanced + maybeExpandIsr(replicaId) + + debug("Recorded replica %d LEO position %d for partition [%s,%d]." + .format(replicaId, offset.messageOffset, topic, partitionId)) + case None => + throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + + " is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId, + offset.messageOffset, assignedReplicas().map(_.brokerId).mkString(","), topic, partitionId)) + } + } + + /** + * Check and maybe expand the ISR of the partition; + * this can happen when a non-ISR replica's LEO has incremented + */ + def maybeExpandIsr(replicaId: Int) { inWriteLock(leaderIsrUpdateLock) { // check if this replica needs to be added to the ISR leaderReplicaIfLocal() match { @@ -252,6 +276,7 @@ class Partition(val topic: String, updateIsr(newInSyncReplicas) replicaManager.isrExpandRate.mark() } + // after the isr change, check if the HW of the partition can now be incremented maybeIncrementLeaderHW(leaderReplica) case None => // nothing to do if no longer leader } @@ -298,8 +323,14 @@ class Partition(val topic: String, } /** - * There is no need to acquire the leaderIsrUpdate lock here since all callers of this private API acquire that lock - * @param leaderReplica + * Check and maybe increment the high watermark of the partition; + * this can happen when + * + * 1. Partition ISR changed + * 2. Leader LEO changed and the ISR is down to 1 + * + * Note There is no need to acquire the leaderIsrUpdate lock here + * since all callers of this private API acquire that lock */ private def maybeIncrementLeaderHW(leaderReplica: Replica) { val allLogEndOffsets = inSyncReplicas.map(_.logEndOffset) @@ -311,7 +342,7 @@ class Partition(val topic: String, // some delayed requests may be unblocked after HW changed val requestKey = new TopicPartitionRequestKey(this.topic, this.partitionId) replicaManager.unblockDelayedFetchRequests(requestKey) - replicaManager.unblockDelayedProduceRequests(requestKey) + replicaManager.tryCompleteDelayedProduce(requestKey) } else { debug("Skipping update high watermark since Old hw %s is larger than new hw %s for partition [%s,%d]. All leo's are %s" .format(oldHighWatermark, newHighWatermark, topic, partitionId, allLogEndOffsets.mkString(","))) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 6e17027..812b445 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -17,30 +17,23 @@ package kafka.server -import kafka.api.{FetchResponsePartitionData, PartitionFetchInfo, FetchResponse, FetchRequest} -import kafka.common.{UnknownTopicOrPartitionException, NotLeaderForPartitionException, TopicAndPartition} +import kafka.api.FetchResponsePartitionData +import kafka.api.PartitionFetchInfo +import kafka.common.UnknownTopicOrPartitionException +import kafka.common.NotLeaderForPartitionException +import kafka.common.TopicAndPartition import scala.collection._ -/** - * A delayed fetch request, which is satisfied (or more - * accurately, unblocked) -- if: - * Case A: This broker is no longer the leader for some partitions it tries to fetch - * - should return whatever data is available for the rest partitions. - * Case B: This broker is does not know of some partitions it tries to fetch - * - should return whatever data is available for the rest partitions. - * Case C: The fetch offset locates not on the last segment of the log - * - should return all the data on that segment. - * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes - * - should return whatever data is available. - */ - case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInfo: PartitionFetchInfo) { override def toString = "[startOffsetMetadata: " + startOffsetMetadata + ", " + "fetchInfo: " + fetchInfo + "]" } +/** + * The fetch metadata maintained by the delayed produce request + */ case class FetchMetadata(fetchMinBytes: Int, fetchOnlyLeader: Boolean, fetchOnlyCommitted: Boolean, @@ -52,13 +45,24 @@ case class FetchMetadata(fetchMinBytes: Int, "partitionStatus: " + fetchPartitionStatus + "]" } - +/** + * A delayed fetch request that can be created by the replica manager and watched + * in the fetch request purgatory + */ class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, callbackOnComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) extends DelayedRequest(delayMs) { + /** + * The request can be completed if: + * Case A: This broker is no longer the leader for some partitions it tries to fetch + * Case B: This broker is does not know of some partitions it tries to fetch + * Case C: The fetch offset locates not on the last segment of the log + * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes + * + */ override def tryComplete() : Boolean = { var accumulatedSize = 0 fetchMetadata.fetchPartitionStatus.foreach { @@ -78,11 +82,12 @@ class DelayedFetch(delayMs: Long, debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) return super.tryComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { - // Case C, this can happen when the folloer replica is lagging too much + // Case C, this can happen when the follower replica is lagging too much debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata)) return super.tryComplete() } else if (fetchOffset.precedes(endOffset)) { - accumulatedSize += endOffset.positionDiff(fetchOffset) + // we need take the partition fetch size as upper bound when accumulating the bytes + accumulatedSize += math.min(endOffset.positionDiff(fetchOffset), fetchStatus.fetchInfo.fetchSize) } } } catch { @@ -102,13 +107,18 @@ class DelayedFetch(delayMs: Long, false } - override def onExpired() { + /** + * Upon expire, complete the fetch request and return + */ + override def expire() { debug("Expire fetch %s and return whatever fetch data is available".format(fetchMetadata)) - completeFetch() + complete() } - def completeFetch() { - // read whatever data is available and return + /** + * Upon completion, read whatever data is available and pass to the complete callback + */ + override def complete() { val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, fetchMetadata.fetchOnlyCommitted, fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 846fc97..bd4d5ad 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -17,22 +17,14 @@ package kafka.server -import kafka.api._ + +import kafka.api.ProducerResponseStatus import kafka.common.ErrorMapping import kafka.common.TopicAndPartition -import kafka.utils.Logging import scala.Some import scala.collection._ -/** A delayed produce request, which is satisfied (or more - * accurately, unblocked) -- if for every partition it produce to: - * Case A: This broker is not the leader: unblock - should return error. - * Case B: This broker is the leader: - * B.1 - If there was a localError (when writing to the local log): unblock - should return error - * B.2 - else, at least requiredAcks replicas should be caught up to this request. - */ - case class ProducePartitionStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { @volatile var acksPending = false @@ -40,6 +32,9 @@ case class ProducePartitionStatus(requiredOffset: Long, responseStatus: Producer .format(acksPending, responseStatus.error, responseStatus.offset, requiredOffset) } +/** + * The produce metadata maintained by the delayed produce request + */ case class ProduceMetadata(produceRequiredAcks: Short, produceStatus: Map[TopicAndPartition, ProducePartitionStatus]) { @@ -47,6 +42,10 @@ case class ProduceMetadata(produceRequiredAcks: Short, .format(produceRequiredAcks, produceStatus) } +/** + * A delayed produce request that can be created by the replica manager and watched + * in the produce request purgatory + */ class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, @@ -66,6 +65,13 @@ class DelayedProduce(delayMs: Long, trace("Initial partition status for %s is %s".format(topicAndPartition, status)) } + /** + * The delayed produce request can be completed if for every partition it produce to: + * Case A: This broker is no longer the leader: should return error + * Case B: This broker is the leader: + * B.1 - If there was a localError (when writing to the local log): should return error + * B.2 - else, at least requiredAcks replicas should be caught up to this request. + */ override def tryComplete(): Boolean = { // check for each partition if it still has pending acks produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => @@ -80,9 +86,11 @@ class DelayedProduce(delayMs: Long, status.requiredOffset, produceMetadata.produceRequiredAcks) case None => + // Case A (false, ErrorMapping.UnknownTopicOrPartitionCode) } if (errorCode != ErrorMapping.NoError) { + // Case B.1 status.acksPending = false status.responseStatus.error = errorCode } else if (hasEnough) { @@ -92,20 +100,25 @@ class DelayedProduce(delayMs: Long, } } - // unblocked if there are no partitions with pending acks + // Case B.2 if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) super.tryComplete() else false } - override def onExpired() { + /** + * Upon expire, complete the produce request and return + */ + override def expire() { debug("Expire produce %s and return the error codes".format(produceMetadata)) - completeProduce() + complete() } - def completeProduce() { - // return the current response status + /** + * Upon completion, return the current response status along with the error code per partition + */ + def complete() { val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) callbackOnComplete(responseStatus) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index b3a5d30..cd862b9 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -118,6 +118,10 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) } + + /** + * Handle an offset commit request + */ def handleOffsetCommitRequest(request: RequestChannel.Request) { val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] @@ -147,7 +151,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /** - * Handle a produce request or offset commit request (which is really a specialized producer request) + * Handle a produce request */ def handleProducerRequest(request: RequestChannel.Request) { val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] @@ -157,7 +161,7 @@ class KafkaApis(val requestChannel: RequestChannel, var errorInResponse = false responseStatus.foreach { case (topicAndPartition, status) => // Here we only print warnings for known errors; if it is unknown, it will cause - // an error message in the replica manager already and hence can be ignored here + // an error message in the replica manager if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { warn("Produce request with correlation id %d from client %s on partition %s failed due to %s" .format(produceRequest.correlationId, produceRequest.clientId, @@ -166,7 +170,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - if(produceRequest.requiredAcks == 0) { + if (produceRequest.requiredAcks == 0) { // no operation needed if producer request.required.acks = 0; however, if there is any error in handling // the request, since no response is expected by the producer, the server will close socket server so that // the producer client will know that some error has happened and will refresh its metadata @@ -189,9 +193,6 @@ class KafkaApis(val requestChannel: RequestChannel, produceRequest.requiredAcks, produceRequest.data, sendResponseCallback) - - // we do not need the data anymore - produceRequest.emptyData() } /** diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 14e0cc6..875ce8a 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -199,15 +199,15 @@ class OffsetManager(val config: OffsetManagerConfig, /** * Store offsets by appending it to the replicated log and then inserting to cache */ + // TODO: generation id and consumer id is needed by coordinator to do consumer checking in the future def storeOffsets(groupName: String, consumerId: String, generationId: Int, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], callbackOnComplete: immutable.Map[TopicAndPartition, Short] => Unit) { - // TODO: generation id and consumer id is needed by coordinator to do consumer checking // first filter out partitions with offset metadata size exceeding limit - // TODO: in the future we may want to only support atomic commit and hence fail the whole commit when this happens + // TODO: in the future we may want to only support atomic commit and hence fail the whole commit var commitStatus = offsetMetadata.mapValues { offsetAndMetadata => if (offsetAndMetadata.metadata != null && offsetAndMetadata.metadata.length() > config.maxMetadataSize) ErrorMapping.OffsetMetadataTooLargeCode @@ -229,7 +229,7 @@ class OffsetManager(val config: OffsetManagerConfig, val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupName)) - val messageSet = Collections.singletonMap(offsetTopicPartition, + val offsetsAndMetadataMessageSet = Collections.singletonMap(offsetTopicPartition, new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)).toMap // set the callback function to insert offsets into cache after log append completed @@ -275,7 +275,7 @@ class OffsetManager(val config: OffsetManagerConfig, replicaManager.appendMessages( config.offsetCommitTimeoutMs.toLong, config.offsetCommitRequiredAcks, - messageSet, + offsetsAndMetadataMessageSet, putCacheCallback) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 4ccd80d..d91ab76 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -98,26 +98,27 @@ class ReplicaManager(val config: KafkaConfig, } /** - * Unblock some delayed produce requests with the request key + * Try to complete some delayed produce requests with the request key; + * this can be triggered when: + * + * 1. The partition HW has changed (for acks = -1). + * 2. A follower replica's fetch operation is received (for acks > 1) */ - def unblockDelayedProduceRequests(key: DelayedRequestKey) { - val satisfied = producerRequestPurgatory.getCompleted(key) - debug("Request key %s unblocked %d producer requests." - .format(key.keyLabel, satisfied.size)) - - // complete the produce operation - satisfied.foreach(_.completeProduce()) + def tryCompleteDelayedProduce(key: DelayedRequestKey) { + val completed = producerRequestPurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d producer requests.".format(key.keyLabel, completed)) } /** - * Unblock some delayed fetch requests with the request key + * Try to complete some delayed fetch requests with the request key; + * this can be triggered when: + * + * 1. The partition HW has changed; + * 2. A new message set is appended to the local log (for follower fetch) */ def unblockDelayedFetchRequests(key: DelayedRequestKey) { - val satisfied = fetchRequestPurgatory.getCompleted(key) - debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, satisfied.size)) - - // complete the fetch operation - satisfied.foreach(_.completeFetch()) + val completed = fetchRequestPurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, completed)) } def startup() { @@ -224,7 +225,7 @@ class ReplicaManager(val config: KafkaConfig, } /** - * Append messages to leader replicas of the partition, and wait for replicated to other replicas, + * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas; * the callback function will be triggered either when timeout or the required acks are satisfied */ def appendMessages(timeout: Long, @@ -253,14 +254,17 @@ class ReplicaManager(val config: KafkaConfig, val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) callbackOnComplete(produceResponseStatus) } else { - // create delayed produce operation and try to watch it in the purgatory + // create delayed produce operation val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, callbackOnComplete) + + // create a list of (topic, partition) pairs to use as keys for this delayed request val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionRequestKey(_)).toSeq - val completedByMe = producerRequestPurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) - if (completedByMe) - delayedProduce.completeProduce() + // try to complete the request immediately, otherwise put it into the purgatory + // this is because while the delayed request is being created, new requests may + // arrive which can make this request completable. + producerRequestPurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) } } @@ -315,7 +319,7 @@ class ReplicaManager(val config: KafkaConfig, } /** - * Fetch messages from the leader replica, + * Fetch messages from the leader replica, and wait until enough data can be fetched and return; * the callback function will be triggered either when timeout or required fetch info is satisfied */ def fetchMessages(timeout: Long, @@ -359,10 +363,10 @@ class ReplicaManager(val config: KafkaConfig, // create a list of (topic, partition) pairs to use as keys for this delayed request val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionRequestKey(_)).toSeq - // add the fetch request for watch if it's not satisfied, otherwise send the response back - val completedByMe = fetchRequestPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) - if (completedByMe) - delayedFetch.completeFetch() + // try to complete the request immediately, otherwise put it into the purgatory; + // this is because while the delayed request is being created, new requests may + // arrive which can make this request completable. + fetchRequestPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) } } @@ -674,35 +678,17 @@ class ReplicaManager(val config: KafkaConfig, private def updateFollowerLEOs(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { debug("Recording follower broker %d log end offsets: %s ".format(replicaId, offsets)) - offsets.foreach { - case (topicAndPartition, offset) => - updateReplicaLEO(topicAndPartition, replicaId, offset) - - // for producer requests with ack > 1, we need to check - // if they can be unblocked after some follower's log end offsets have moved - unblockDelayedProduceRequests(new TopicPartitionRequestKey(topicAndPartition)) - } - } - - private def updateReplicaLEO(topicAndPartition: TopicAndPartition, replicaId: Int, offset: LogOffsetMetadata) = { - getPartition(topicAndPartition.topic, topicAndPartition.partition) match { - case Some(partition) => - partition.getReplica(replicaId) match { - case Some(replica) => - replica.logEndOffset = offset - - // check if we need to update HW and expand Isr after some of its replica's LEOs have changed - partition.updateLeaderHWAndMaybeExpandIsr(replicaId) - - debug("Recorded replica %d LEO position %d for partition %s.".format(replicaId, offset.messageOffset, topicAndPartition)) - case None => - throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + - " is not recognized to be one of the assigned replicas %s for partition %s").format(localBrokerId, replicaId, - offset.messageOffset, partition.assignedReplicas().map(_.brokerId).mkString(","), topicAndPartition)) - - } - case None => - warn("While recording the replica LEO, the partition %s hasn't been created.".format(topicAndPartition)) + offsets.foreach { case (topicAndPartition, offset) => + getPartition(topicAndPartition.topic, topicAndPartition.partition) match { + case Some(partition) => + partition.updateReplicaLEO(replicaId, offset) + + // for producer requests with ack > 1, we need to check + // if they can be unblocked after some follower's log end offsets have moved + tryCompleteDelayedProduce(new TopicPartitionRequestKey(topicAndPartition)) + case None => + warn("While recording the replica LEO, the partition %s hasn't been created.".format(topicAndPartition)) + } } } @@ -728,8 +714,10 @@ class ReplicaManager(val config: KafkaConfig, } def shutdown() { - info("Shut down") + info("Shutting down") replicaFetcherManager.shutdown() + fetchRequestPurgatory.shutdown() + producerRequestPurgatory.shutdown() checkpointHighWatermarks() info("Shut down completely") } diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index ee257de..e0c2882 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -29,22 +29,28 @@ import com.yammer.metrics.core.Gauge /** - * An operation whose processing needs to be delayed for at most the given delayMs; - * upon complete, the given callback function will be triggered. For example a delayed - * message append operation could be waiting for specified number of acks; or a delayed - * message fetch operation could be waiting for a given number of bytes to accumulate. + * An operation whose processing needs to be delayed for at most the given delayMs. For example + * a delayed produce operation could be waiting for specified number of acks; or + * a delayed fetch operation could be waiting for a given number of bytes to accumulate. */ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { val completed = new AtomicBoolean(false) /* - * Check if the delayed operation can be completed + * Check if the delayed operation can be completed by the caller * * Note that concurrent threads can check if an operation can be completed or not, * but only the first thread will succeed in completing the operation and return * true, others will still return false */ - def tryComplete(): Boolean = completed.compareAndSet(false, true) + def tryComplete(): Boolean = { + if (completed.compareAndSet(false, true)) { + complete() + true + } else { + false + } + } /** * Check if the delayed operation is already completed @@ -52,15 +58,18 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { def isCompleted(): Boolean = completed.get() /* - * When delayMs has elapsed, expire the delayed operation + * Process for expiring a timed out request */ - def onExpired(): Unit + def expire(): Unit = complete + /** + * Process for completing a request + */ + def complete(): Unit } /** * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. - * */ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: Int = 1000) extends Logging with KafkaMetricsGroup { @@ -106,14 +115,16 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In if (operation.isCompleted()) return false val watchers = watchersFor(key) - // if the operation is completed by myself, stop adding it to + // if the operation can by completed by myself, stop adding it to // any further lists and return true immediately - if(! watchers.checkAndMaybeAdd(operation)) { + if(operation synchronized operation.tryComplete()) { return true + } else { + watchers.watch(operation) } } - // if it is indeed watched, add to the expire queue also + // if it cannot be completed by now and hence is watched, add to the expire queue also if (! operation.isCompleted()) { expirationReaper.enqueue(operation) } @@ -122,16 +133,17 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In } /** - * Return a list of completed operations with the given watch key. + * Check if some some delayed requests can be completed with the given watch key, + * and if yes complete them. * - * @return the list of completed operations + * @return the number of completed requests during this process */ - def getCompleted(key: Any): Seq[T] = { + def checkAndComplete(key: Any): Int = { val watchers = watchersForKey.get(key) if(watchers == null) - Seq.empty + 0 else - watchers.collectCompletedOperations() + watchers.completeWatched() } /* @@ -145,7 +157,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In * may be larger than the number of real operations watched */ protected def size() = watchersForKey.values.map(_.watched).sum + expirationReaper.enqueued - + /** * Shutdown the expire reaper thread */ @@ -161,55 +173,48 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In def watched = requests.size() - // potentially add the element to watch if it is not satisfied yet - def checkAndMaybeAdd(t: T): Boolean = { + // add the element to watch + def watch(t: T) { synchronized { - // if the operation can be completed, return false; otherwise add to watch list - if(t.tryComplete()) { - return false - } else { - requests.add(t) - return true - } + requests.add(t) } } - // traverse the list and purge satisfied elements - def purgeSatisfied(): Int = { + // traverse the list and try to complete some watched elements + def completeWatched(): Int = { + var completed = 0 synchronized { val iter = requests.iterator() - var purged = 0 - while (iter.hasNext) { + while(iter.hasNext) { val curr = iter.next - if(curr.isCompleted()) { + if (curr.isCompleted()) { + // another thread has completed this request, just remove it iter.remove() - purged += 1 + } else { + if(curr.tryComplete()) { + iter.remove() + completed += 1 + } } } - purged } + completed } - // traverse the list and try to satisfy watched elements - def collectCompletedOperations(): Seq[T] = { - val response = new mutable.ArrayBuffer[T] + // traverse the list and purge elements that are already completed by others + private def purgeCompleted(): Int = { + var purged = 0 synchronized { val iter = requests.iterator() - while(iter.hasNext) { + while (iter.hasNext) { val curr = iter.next - if (curr.isCompleted()) { - // another thread has completed this request, just remove it + if(curr.isCompleted()) { iter.remove() - } else { - val completed = curr.tryComplete() - if(completed) { - iter.remove() - response += curr - } + purged += 1 } } } - response + purged } } @@ -273,15 +278,17 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In override def doWork() { + // try to get the next expired operation and trigger its expiration process val curr = pollExpired() if (curr != null.asInstanceOf[T]) { - curr.onExpired() + curr.expire() } - if (size() >= purgeInterval) { // see if we need to force a full purge + // see if we need to force a full purge + if (size() >= purgeInterval) { debug("Beginning purgatory purge") val purged = purgeSatisfied() debug("Purged %d operations from delay queue.".format(purged)) - val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum + val numPurgedFromWatchers = watchersForKey.values.map(_.purgeCompleted()).sum debug("Purged %d operations from watch lists.".format(numPurgedFromWatchers)) } } diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 5ebd585..ad77ae8 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -191,7 +191,7 @@ class SimpleFetchTest extends JUnit3Suite { val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) }).anyTimes() - EasyMock.expect(replicaManager.unblockDelayedProduceRequests(EasyMock.anyObject())).anyTimes() + EasyMock.expect(replicaManager.tryCompleteDelayedProduce(EasyMock.anyObject())).anyTimes() EasyMock.replay(replicaManager) val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) -- 1.7.12.4 From b0971596425291eaadad9fa792e105527433f489 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 21 Aug 2014 11:30:27 -0700 Subject: [PATCH 16/33] minor --- core/src/main/scala/kafka/server/RequestPurgatory.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index e0c2882..e6aca51 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -202,7 +202,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In } // traverse the list and purge elements that are already completed by others - private def purgeCompleted(): Int = { + def purgeCompleted(): Int = { var purged = 0 synchronized { val iter = requests.iterator() -- 1.7.12.4 From 63e196e614e6d5020040036b450db9a1437082ab Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 21 Aug 2014 16:47:50 -0700 Subject: [PATCH 17/33] unit test fix 1 --- .../main/scala/kafka/server/RequestPurgatory.scala | 10 +- .../unit/kafka/server/RequestPurgatoryTest.scala | 85 ++++--- .../scala/unit/kafka/server/SimpleFetchTest.scala | 262 ++++++--------------- 3 files changed, 131 insertions(+), 226 deletions(-) diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index e6aca51..6efa96a 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -147,17 +147,17 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In } /* - * Return the watch list of the given key - */ - private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) - - /* * Return the size of the purgatory, which is size of watch lists plus the size of the expire reaper. * Since an operation may still be in the watch lists even when it has been completed, this number * may be larger than the number of real operations watched */ protected def size() = watchersForKey.values.map(_.watched).sum + expirationReaper.enqueued + /* + * Return the watch list of the given key + */ + private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) + /** * Shutdown the expire reaper thread */ diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala index bdeed80..a1d8927 100644 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala @@ -17,28 +17,25 @@ package kafka.server -import scala.collection._ import org.junit.Test -import junit.framework.Assert._ -import kafka.message._ -import kafka.api._ -import kafka.utils.TestUtils import org.scalatest.junit.JUnit3Suite - +import junit.framework.Assert._ class RequestPurgatoryTest extends JUnit3Suite { - val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes))) - val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes))) - var purgatory: MockDelayedOperationPurgatory = null + var purgatory: RequestPurgatory[MockDelayedRequest] = null override def setUp() { super.setUp() +<<<<<<< HEAD <<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala purgatory = new MockRequestPurgatory(5) ======= purgatory = new MockDelayedOperationPurgatory() >>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala +======= + purgatory = new RequestPurgatory[MockDelayedRequest]() +>>>>>>> unit test fix 1 } override def tearDown() { @@ -48,36 +45,37 @@ class RequestPurgatoryTest extends JUnit3Suite { @Test def testRequestSatisfaction() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r2 = new DelayedRequest(Array("test2"), null, 100000L) - assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.update("test1").size) - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test1").size) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test2").size) - purgatory.satisfied += r1 - assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1")) - assertEquals("Nothing satisfied", 0, purgatory.update("test1").size) - purgatory.satisfied += r2 - assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2")) - assertEquals("Nothing satisfied", 0, purgatory.update("test2").size) + val r1 = new MockDelayedRequest(100000L) + val r2 = new MockDelayedRequest(100000L) + assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.checkAndComplete("test1")) + assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) + assertEquals("Still nothing satisfied", 0, purgatory.checkAndComplete("test1")) + assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) + assertEquals("Still nothing satisfied", 0, purgatory.checkAndComplete("test2")) + r1.completable = true + assertEquals("r1 satisfied", 1, purgatory.checkAndComplete("test1")) + assertEquals("Nothing satisfied", 0, purgatory.checkAndComplete("test1")) + r2.completable = true + assertEquals("r2 satisfied", 1, purgatory.checkAndComplete("test2")) + assertEquals("Nothing satisfied", 0, purgatory.checkAndComplete("test2")) } @Test def testRequestExpiry() { val expiration = 20L - val r1 = new DelayedRequest(Array("test1"), null, expiration) - val r2 = new DelayedRequest(Array("test1"), null, 200000L) + val r1 = new MockDelayedRequest(expiration) + val r2 = new MockDelayedRequest(200000L) val start = System.currentTimeMillis - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - purgatory.awaitExpiration(r1) + assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) + assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) + r1.awaitExpiration() val elapsed = System.currentTimeMillis - start - assertTrue("r1 expired", purgatory.expired.contains(r1)) - assertTrue("r2 hasn't expired", !purgatory.expired.contains(r2)) + assertTrue("r1 expired", r1.expired) + assertFalse("r2 hasn't expired", r2.expired) assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) } +<<<<<<< HEAD @Test def testRequestPurge() { val r1 = new DelayedRequest(Array("test1"), null, 100000L) @@ -121,15 +119,34 @@ class RequestPurgatoryTest extends JUnit3Suite { def awaitExpiration(delayed: DelayedRequest) = { delayed synchronized { delayed.wait() +======= + // A mock delayed request that can be completed / expired at will + class MockDelayedRequest(delayMs: Long) extends DelayedRequest(delayMs) { + var completable = false + var expired = false + + def awaitExpiration() { + synchronized { + wait() +>>>>>>> unit test fix 1 } } - def checkSatisfied(delayed: DelayedRequest): Boolean = satisfied.contains(delayed) - def expire(delayed: DelayedRequest) { - expired += delayed - delayed synchronized { - delayed.notify() + + override def tryComplete() = { + if (completable) + completed.compareAndSet(false, true) + else + false + } + + override def expire() { + synchronized { + expired = true + notify() } } + + override def complete() { /* do nothing */ } } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index ad77ae8..09cc0c1 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -17,18 +17,20 @@ package kafka.server import kafka.api._ -import kafka.cluster.{Partition, Replica} -import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.utils._ +import kafka.cluster.Replica +import kafka.common.TopicAndPartition import kafka.log.Log import kafka.message.{ByteBufferMessageSet, Message} -import kafka.network.RequestChannel -import kafka.utils.{ZkUtils, Time, TestUtils, MockTime} import scala.Some +import java.util.Collections +import collection.JavaConversions._ import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ class SimpleFetchTest extends JUnit3Suite { @@ -37,213 +39,99 @@ class SimpleFetchTest extends JUnit3Suite { override val replicaFetchWaitMaxMs = 100 override val replicaLagMaxMessages = 10L }) - val topic = "foo" + + // set the replica manager with the partition + val time = new MockTime + val leaderLEO = 20L + val followerLEO = 15L + val partitionHW = 5 + + val fetchSize = 100 + val messagesToHW = new Message("messageToHW".getBytes()) + val messagesToLEO = new Message("messageToLEO".getBytes()) + + val topic = "test-topic" val partitionId = 0 + val topicAndPartition = TopicAndPartition(topic, partitionId) - /** - * The scenario for this test is that there is one topic, "test-topic", one broker "0" that has - * one partition with one follower replica on broker "1". The leader replica on "0" - * has HW of "5" and LEO of "20". The follower on broker "1" has a local replica - * with a HW matching the leader's ("5") and LEO of "15", meaning it's not in-sync - * but is still in ISR (hasn't yet expired from ISR). - * - * When a normal consumer fetches data, it should only see data up to the HW of the leader, - * in this case up an offset of "5". - */ - def testNonReplicaSeesHwWhenFetching() { - /* setup */ - val time = new MockTime - val leo = 20L - val hw = 5 - val fetchSize = 100 - val messages = new Message("test-message".getBytes()) + val fetchInfo = Collections.singletonMap(topicAndPartition, PartitionFetchInfo(0, fetchSize)).toMap + + var replicaManager: ReplicaManager = null + + override def setUp() { + super.setUp() // create nice mock since we don't particularly care about zkclient calls val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) - EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) + //EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) EasyMock.replay(zkClient) - val log = EasyMock.createMock(classOf[kafka.log.Log]) - EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() - EasyMock.expect(log) - EasyMock.expect(log.read(0, fetchSize, Some(hw))).andReturn( + // create nice mock since we don't particularly care about scheduler calls + val scheduler = EasyMock.createNiceMock(classOf[KafkaScheduler]) + EasyMock.replay(scheduler) + + // create the log which takes read with either HW max offset or none max offset + val log = EasyMock.createMock(classOf[Log]) + EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes() + //EasyMock.expect(log) + EasyMock.expect(log.read(0, fetchSize, Some(partitionHW))).andReturn( + new FetchDataInfo( + new LogOffsetMetadata(0L, 0L, 0), + new ByteBufferMessageSet(messagesToHW) + )).anyTimes() + EasyMock.expect(log.read(0, fetchSize, None)).andReturn( new FetchDataInfo( - new LogOffsetMetadata(0L, 0L, leo.toInt), - new ByteBufferMessageSet(messages) + new LogOffsetMetadata(0L, 0L, 0), + new ByteBufferMessageSet(messagesToLEO) )).anyTimes() EasyMock.replay(log) + // create the log manager that is aware of this mock log val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() + EasyMock.expect(logManager.getLog(topicAndPartition)).andReturn(Some(log)).anyTimes() EasyMock.replay(logManager) - val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(0, fetchSize, Some(hw)) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(configs(1).brokerId).get.logEndOffset = new LogOffsetMetadata(leo - 5L, 0L, leo.toInt - 5) - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(0, fetchSize, Some(hw)) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - - EasyMock.replay(replicaManager) - - val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) - - val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) - - // start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary) - // don't provide replica or leader callbacks since they will not be tested here - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) - - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.addOrUpdatePartitionInfo(topic, partitionId, partitionStateInfo) - EasyMock.replay(partitionStateInfo) - // This request (from a follower) wants to read up to 2*HW but should only get back up to HW bytes into the log - val goodFetch = new FetchRequestBuilder() - .replicaId(Request.OrdinaryConsumerId) - .addFetch(topic, partitionId, 0, fetchSize) - .build() - val goodFetchBB = TestUtils.createRequestByteBuffer(goodFetch) - - // send the request - apis.handleFetchRequest(new RequestChannel.Request(processor=1, requestKey=5, buffer=goodFetchBB, startTimeMs=1)) - - // make sure the log only reads bytes between 0->HW (5) - EasyMock.verify(log) + // create the replica manager + replicaManager = new ReplicaManager(configs.head, time, zkClient, scheduler, logManager, AtomicBoolean(false)) + + // add the partition with two replicas, both in ISR + val partition = replicaManager.getOrCreatePartition(topic, partitionId) + + // create the leader replica with the local log + val leaderReplica = new Replica(configs(0).brokerId, partition, time, 0, Some(log)) + leaderReplica.highWatermark = new LogOffsetMetadata(partitionHW) + partition.leaderReplicaIdOpt = Some(leaderReplica.brokerId) + + // create the follower replica with defined log end offset + val followerReplica= new Replica(configs(1).brokerId, partition, time) + followerReplica.logEndOffset = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt) + + // add both of them to ISR + val allReplicas = List(leaderReplica, followerReplica) + allReplicas.foreach(partition.addReplicaIfNotExists(_)) + partition.inSyncReplicas = allReplicas.toSet + } + + override def tearDown() { + super.tearDown() } /** - * The scenario for this test is that there is one topic, "test-topic", on broker "0" that has + * The scenario for this test is that there is one topic, "test-topic", one broker "0" that has * one partition with one follower replica on broker "1". The leader replica on "0" * has HW of "5" and LEO of "20". The follower on broker "1" has a local replica * with a HW matching the leader's ("5") and LEO of "15", meaning it's not in-sync * but is still in ISR (hasn't yet expired from ISR). * - * When the follower from broker "1" fetches data, it should see data upto the log end offset ("20") + * When a normal consumer fetches data, it should only see data up to the HW of the leader, + * in this case up an offset of "5". */ - def testReplicaSeesLeoWhenFetching() { - /* setup */ - val time = new MockTime - val leo = 20 - val hw = 5 - - val messages = new Message("test-message".getBytes()) - - val followerReplicaId = configs(1).brokerId - val followerLEO = 15 - - val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) - EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) - EasyMock.replay(zkClient) - - val log = EasyMock.createMock(classOf[kafka.log.Log]) - EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() - EasyMock.expect(log.read(followerLEO, Integer.MAX_VALUE, None)).andReturn( - new FetchDataInfo( - new LogOffsetMetadata(followerLEO, 0L, followerLEO), - new ByteBufferMessageSet(messages) - )).anyTimes() - EasyMock.replay(log) - - val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, 0))).andReturn(Some(log)).anyTimes() - EasyMock.replay(logManager) - - val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(followerLEO, Integer.MAX_VALUE, None) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(followerReplicaId).get.logEndOffset = new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, followerLEO) - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.updateReplicaLEO(TopicAndPartition(topic, partitionId), followerReplicaId, new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, followerLEO))) - EasyMock.expect(replicaManager.getReplica(topic, partitionId, followerReplicaId)).andReturn(partition.inSyncReplicas.find(_.brokerId == configs(1).brokerId)) - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(followerLEO, Integer.MAX_VALUE, None) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.expect(replicaManager.tryCompleteDelayedProduce(EasyMock.anyObject())).anyTimes() - EasyMock.replay(replicaManager) - - val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) - - val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) - - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.addOrUpdatePartitionInfo(topic, partitionId, partitionStateInfo) - EasyMock.replay(partitionStateInfo) - - /** - * This fetch, coming from a replica, requests all data at offset "15". Because the request is coming - * from a follower, the leader should oblige and read beyond the HW. - */ - val bigFetch = new FetchRequestBuilder() - .replicaId(followerReplicaId) - .addFetch(topic, partitionId, followerLEO, Integer.MAX_VALUE) - .build() - - val fetchRequestBB = TestUtils.createRequestByteBuffer(bigFetch) - - // send the request - apis.handleFetchRequest(new RequestChannel.Request(processor=0, requestKey=5, buffer=fetchRequestBB, startTimeMs=1)) - - /** - * Make sure the log satisfies the fetch from a follower by reading data beyond the HW, mainly all bytes after - * an offset of 15 - */ - EasyMock.verify(log) - } - - private def getPartitionWithAllReplicasInISR(topic: String, partitionId: Int, time: Time, leaderId: Int, - localLog: Log, leaderHW: Long, replicaManager: ReplicaManager): Partition = { - val partition = new Partition(topic, partitionId, time, replicaManager) - val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) + def testReadFromLog() { - val allReplicas = getFollowerReplicas(partition, leaderId, time) :+ leaderReplica - allReplicas.foreach(partition.addReplicaIfNotExists(_)) - // set in sync replicas for this partition to all the assigned replicas - partition.inSyncReplicas = allReplicas.toSet - // set the leader and its hw and the hw update time - partition.leaderReplicaIdOpt = Some(leaderId) - leaderReplica.highWatermark = new LogOffsetMetadata(leaderHW) - partition - } + assertEquals("", messagesToHW, replicaManager.readFromLocalLog(true, true, fetchInfo) + .get(topicAndPartition).get.info.messageSet) - private def getFollowerReplicas(partition: Partition, leaderId: Int, time: Time): Seq[Replica] = { - configs.filter(_.brokerId != leaderId).map { config => - new Replica(config.brokerId, partition, time) - } + assertEquals("", messagesToLEO, replicaManager.readFromLocalLog(true, false, fetchInfo) + .get(topicAndPartition).get.info.messageSet) } - } -- 1.7.12.4 From 57417e3adabab15a91d4518d6260f18d3bfab51c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 22 Aug 2014 10:31:17 -0700 Subject: [PATCH 18/33] Fixed RequestPurgatoryTest --- core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 09cc0c1..998b6c5 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -25,6 +25,7 @@ import kafka.message.{ByteBufferMessageSet, Message} import scala.Some import java.util.Collections +import java.util.concurrent.atomic.AtomicBoolean import collection.JavaConversions._ import org.easymock.EasyMock @@ -92,7 +93,7 @@ class SimpleFetchTest extends JUnit3Suite { EasyMock.replay(logManager) // create the replica manager - replicaManager = new ReplicaManager(configs.head, time, zkClient, scheduler, logManager, AtomicBoolean(false)) + replicaManager = new ReplicaManager(configs.head, time, zkClient, scheduler, logManager, new AtomicBoolean(false)) // add the partition with two replicas, both in ISR val partition = replicaManager.getOrCreatePartition(topic, partitionId) -- 1.7.12.4 From cd6fee431d95ddd48494a7f46fd63c4e99dc6499 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 22 Aug 2014 14:29:59 -0700 Subject: [PATCH 19/33] fix 2 --- core/src/main/scala/kafka/server/DelayedProduce.scala | 10 +++++++--- core/src/main/scala/kafka/server/ReplicaManager.scala | 4 ++-- core/src/test/resources/log4j.properties | 6 +++--- core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala | 4 ++-- 4 files changed, 14 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index bd4d5ad..515a752 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -53,7 +53,7 @@ class DelayedProduce(delayMs: Long, extends DelayedRequest(delayMs) { // first update the acks pending variable according to the error code - produceMetadata.produceStatus foreach { case (topicAndPartition, status) => + produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => if (status.responseStatus.error == ErrorMapping.NoError) { // Timeout error state will be cleared when required acks are received status.acksPending = true @@ -65,6 +65,10 @@ class DelayedProduce(delayMs: Long, trace("Initial partition status for %s is %s".format(topicAndPartition, status)) } + produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => + trace("Debugging: initial partition status for %s is %s".format(topicAndPartition, status)) + } + /** * The delayed produce request can be completed if for every partition it produce to: * Case A: This broker is no longer the leader: should return error @@ -75,8 +79,8 @@ class DelayedProduce(delayMs: Long, override def tryComplete(): Boolean = { // check for each partition if it still has pending acks produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => - trace("Checking produce satisfaction for %s, acksPending = %b" - .format(topicAndPartition, status.acksPending)) + trace("Checking produce satisfaction for %s, current status %s" + .format(topicAndPartition, status)) // skip those partitions that have already been satisfied if (status.acksPending) { val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index d91ab76..5562a42 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -390,9 +390,9 @@ class ReplicaManager(val config: KafkaConfig, // decide whether to only fetch committed data (i.e. messages below high watermark) val maxOffsetOpt = if (readOnlyCommitted) - None - else Some(localReplica.highWatermark.messageOffset) + else + None // read on log val logReadInfo = localReplica.log match { diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 1b7d5d8..9973dad 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -log4j.rootLogger=OFF, stdout +log4j.rootLogger=TRACE, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka=ERROR +log4j.logger.kafka=TRACE +log4j.logger.org.apache.kafka=TRACE # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 998b6c5..f161f33 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -130,9 +130,9 @@ class SimpleFetchTest extends JUnit3Suite { def testReadFromLog() { assertEquals("", messagesToHW, replicaManager.readFromLocalLog(true, true, fetchInfo) - .get(topicAndPartition).get.info.messageSet) + .get(topicAndPartition).get.info.messageSet.head.message) assertEquals("", messagesToLEO, replicaManager.readFromLocalLog(true, false, fetchInfo) - .get(topicAndPartition).get.info.messageSet) + .get(topicAndPartition).get.info.messageSet.head.message) } } -- 1.7.12.4 From 80b7efe24a8d34c8a292925f3bf5054ec8b170e1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 25 Aug 2014 16:07:41 -0700 Subject: [PATCH 20/33] fix mapValues to map --- core/src/main/scala/kafka/server/ReplicaManager.scala | 11 ++++++----- core/src/test/resources/log4j.properties | 6 +++--- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 5562a42..a76216c 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -237,11 +237,12 @@ class ReplicaManager(val config: KafkaConfig, val localProduceResults = appendToLocalLog(messagesPerPartition) debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) - val produceStatus = localProduceResults.mapValues(result => - ProducePartitionStatus( - result.info.lastOffset + 1, // required offset - ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status - ) + val produceStatus = localProduceResults.map{ case (topicAndPartition, result) => + topicAndPartition -> + ProducePartitionStatus( + result.info.lastOffset + 1, // required offset + ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status + } if(requiredAcks == 0) { // if required acks = 0 we can trigger complete immediately diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 9973dad..1b7d5d8 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -log4j.rootLogger=TRACE, stdout +log4j.rootLogger=OFF, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=TRACE -log4j.logger.org.apache.kafka=TRACE +log4j.logger.kafka=ERROR +log4j.logger.org.apache.kafka=ERROR # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN -- 1.7.12.4 From cab59689eee62d7056c57d71d8215b33d8dadd47 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 25 Aug 2014 17:24:28 -0700 Subject: [PATCH 21/33] incorporate Jun's comments phase 1 --- core/src/main/scala/kafka/cluster/Partition.scala | 15 +++--- .../src/main/scala/kafka/server/DelayedFetch.scala | 2 +- .../main/scala/kafka/server/DelayedProduce.scala | 7 ++- core/src/main/scala/kafka/server/KafkaApis.scala | 4 ++ .../main/scala/kafka/server/OffsetManager.scala | 10 ++-- .../main/scala/kafka/server/ReplicaManager.scala | 54 +++++++++++----------- .../main/scala/kafka/server/RequestPurgatory.scala | 19 ++++---- 7 files changed, 61 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index a4a4c2f..7487a5e 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -238,7 +238,8 @@ class Partition(val topic: String, case Some(replica) => replica.logEndOffset = offset - // check if we need to expand Isr to include this replica after its LEO has advanced + // check if we need to expand ISR to include this replica + // if it is not in the ISR yet maybeExpandIsr(replicaId) debug("Recorded replica %d LEO position %d for partition [%s,%d]." @@ -252,7 +253,7 @@ class Partition(val topic: String, /** * Check and maybe expand the ISR of the partition; - * this can happen when a non-ISR replica's LEO has incremented + * this can happen when a replica's LEO has incremented */ def maybeExpandIsr(replicaId: Int) { inWriteLock(leaderIsrUpdateLock) { @@ -276,8 +277,10 @@ class Partition(val topic: String, updateIsr(newInSyncReplicas) replicaManager.isrExpandRate.mark() } - // after the isr change, check if the HW of the partition can now be incremented + // check if the HW of the partition can now be incremented + // since the replica maybe now in the ISR and its LEO has just incremented maybeIncrementLeaderHW(leaderReplica) + case None => // nothing to do if no longer leader } } @@ -327,7 +330,7 @@ class Partition(val topic: String, * this can happen when * * 1. Partition ISR changed - * 2. Leader LEO changed and the ISR is down to 1 + * 2. Any replica's LEO changed (e.g. leader LEO changed and the ISR is down to 1) * * Note There is no need to acquire the leaderIsrUpdate lock here * since all callers of this private API acquire that lock @@ -341,7 +344,7 @@ class Partition(val topic: String, debug("High watermark for partition [%s,%d] updated to %s".format(topic, partitionId, newHighWatermark)) // some delayed requests may be unblocked after HW changed val requestKey = new TopicPartitionRequestKey(this.topic, this.partitionId) - replicaManager.unblockDelayedFetchRequests(requestKey) + replicaManager.tryCompleteDelayedFetch(requestKey) replicaManager.tryCompleteDelayedProduce(requestKey) } else { debug("Skipping update high watermark since Old hw %s is larger than new hw %s for partition [%s,%d]. All leo's are %s" @@ -410,7 +413,7 @@ class Partition(val topic: String, val info = log.append(messages, assignOffsets = true) // probably unblock some follower fetch requests since log end offset has been updated - replicaManager.unblockDelayedFetchRequests(new TopicPartitionRequestKey(this.topic, this.partitionId)) + replicaManager.tryCompleteDelayedFetch(new TopicPartitionRequestKey(this.topic, this.partitionId)) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(leaderReplica) info diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 812b445..c30cc0a 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -57,11 +57,11 @@ class DelayedFetch(delayMs: Long, /** * The request can be completed if: + * * Case A: This broker is no longer the leader for some partitions it tries to fetch * Case B: This broker is does not know of some partitions it tries to fetch * Case C: The fetch offset locates not on the last segment of the log * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes - * */ override def tryComplete() : Boolean = { var accumulatedSize = 0 diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 515a752..e48a45c 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -70,7 +70,9 @@ class DelayedProduce(delayMs: Long, } /** - * The delayed produce request can be completed if for every partition it produce to: + * The delayed produce request can be completed if every partition + * it produces to is satisfied by one of the following: + * * Case A: This broker is no longer the leader: should return error * Case B: This broker is the leader: * B.1 - If there was a localError (when writing to the local log): should return error @@ -98,13 +100,14 @@ class DelayedProduce(delayMs: Long, status.acksPending = false status.responseStatus.error = errorCode } else if (hasEnough) { + // Case B.2 status.acksPending = false status.responseStatus.error = ErrorMapping.NoError } } } - // Case B.2 + // check if each partition has satisfied either case A or case B if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) super.tryComplete() else diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index cd862b9..932492f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -211,6 +211,10 @@ class KafkaApis(val requestChannel: RequestChannel, .format(fetchRequest.correlationId, fetchRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(data.error))) } + + // record the bytes out metrics only when the response is being sent + BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(data.messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) } val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 875ce8a..d7d060f 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -204,7 +204,7 @@ class OffsetManager(val config: OffsetManagerConfig, consumerId: String, generationId: Int, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], - callbackOnComplete: immutable.Map[TopicAndPartition, Short] => Unit) { + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { // first filter out partitions with offset metadata size exceeding limit // TODO: in the future we may want to only support atomic commit and hence fail the whole commit @@ -229,8 +229,8 @@ class OffsetManager(val config: OffsetManagerConfig, val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupName)) - val offsetsAndMetadataMessageSet = Collections.singletonMap(offsetTopicPartition, - new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)).toMap + 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]) { @@ -241,7 +241,7 @@ class OffsetManager(val config: OffsetManagerConfig, // construct the commit response status and insert // the offset and metadata to cache iff the append status has no error - val status = responseStatus.get(offsetTopicPartition).get + val status = responseStatus(offsetTopicPartition) if (status.error == ErrorMapping.NoError) { filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => @@ -268,7 +268,7 @@ class OffsetManager(val config: OffsetManagerConfig, } } - callbackOnComplete(commitStatus) + responseCallback(commitStatus) } // call replica manager to append the offset messages diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index a76216c..d68bd7d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -116,7 +116,7 @@ class ReplicaManager(val config: KafkaConfig, * 1. The partition HW has changed; * 2. A new message set is appended to the local log (for follower fetch) */ - def unblockDelayedFetchRequests(key: DelayedRequestKey) { + def tryCompleteDelayedFetch(key: DelayedRequestKey) { val completed = fetchRequestPurgatory.checkAndComplete(key) debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, completed)) } @@ -196,7 +196,7 @@ class ReplicaManager(val config: KafkaConfig, def getReplicaOrException(topic: String, partition: Int): Replica = { val replicaOpt = getReplica(topic, partition) if(replicaOpt.isDefined) - replicaOpt.get + return replicaOpt.get else throw new ReplicaNotAvailableException("Replica %d is not available for partition [%s,%d]".format(config.brokerId, topic, partition)) } @@ -231,7 +231,7 @@ class ReplicaManager(val config: KafkaConfig, def appendMessages(timeout: Long, requiredAcks: Short, messagesPerPartition: Map[TopicAndPartition, MessageSet], - callbackOnComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { val sTime = SystemTime.milliseconds val localProduceResults = appendToLocalLog(messagesPerPartition) @@ -244,20 +244,21 @@ class ReplicaManager(val config: KafkaConfig, ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status } - if(requiredAcks == 0) { - // if required acks = 0 we can trigger complete immediately - val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) - callbackOnComplete(produceResponseStatus) - } else if (requiredAcks == 1 || + if(requiredAcks == 0 || + requiredAcks == 1 || messagesPerPartition.size <= 0 || localProduceResults.values.count(_.error.isDefined) == messagesPerPartition.size) { - // if required acks = 1 or all partition appends have failed we can trigger complete immediately + // in case of the following we can respond immediately: + // + // 1. required acks = 0 or 1 + // 2. there is no data to append + // 3. all partition appends have failed val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) - callbackOnComplete(produceResponseStatus) + responseCallback(produceResponseStatus) } else { // create delayed produce operation val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) - val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, callbackOnComplete) + val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) // create a list of (topic, partition) pairs to use as keys for this delayed request val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionRequestKey(_)).toSeq @@ -327,13 +328,13 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchMinBytes: Int, fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], - callbackOnComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { - val fetchOnlyLeader: Boolean = replicaId != Request.DebuggingConsumerId + val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) // read from local logs - val logReadResults = readFromLocalLog(fetchOnlyLeader, fetchOnlyCommitted, fetchInfo) + val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchInfo) // if the fetch comes from the follower, // update its corresponding log end offset @@ -345,21 +346,21 @@ class ReplicaManager(val config: KafkaConfig, val errorReadingData = logReadResults.values.foldLeft(false) ((errorIncurred, readResult) => errorIncurred || (readResult.errorCode != ErrorMapping.NoError)) - // send the data immediately if 1) fetch request does not want to wait - // 2) fetch request does not require any data - // 3) has enough data to respond - // 4) some error happens while reading data + // respond immediately if 1) fetch request does not want to wait + // 2) fetch request does not require any data + // 3) has enough data to respond + // 4) some error happens while reading data if(timeout <= 0 || fetchInfo.size <= 0 || bytesReadable >= fetchMinBytes || errorReadingData) { val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - callbackOnComplete(fetchPartitionData) + responseCallback(fetchPartitionData) } else { // construct the fetch results from the read results val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => (topicAndPartition, FetchPartitionStatus(result.info.fetchOffset, fetchInfo.get(topicAndPartition).get)) } - val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyLeader, fetchOnlyCommitted, fetchPartitionStatus) - val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, callbackOnComplete) + val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyFromLeader, fetchOnlyCommitted, fetchPartitionStatus) + val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, responseCallback) // create a list of (topic, partition) pairs to use as keys for this delayed request val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionRequestKey(_)).toSeq @@ -374,7 +375,7 @@ class ReplicaManager(val config: KafkaConfig, /** * Read from a single topic/partition at the given offset upto maxSize bytes */ - def readFromLocalLog(readOnlyIfLeader: Boolean, + def readFromLocalLog(fetchOnlyFromLeader: Boolean, readOnlyCommitted: Boolean, readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { @@ -384,7 +385,7 @@ class ReplicaManager(val config: KafkaConfig, trace("Fetching log segment for topic %s, partition %d, offset %d, size %d".format(topic, partition, offset, fetchSize)) // decide whether to only fetch from leader - val localReplica = if (readOnlyIfLeader) + val localReplica = if (fetchOnlyFromLeader) getLeaderReplicaIfLocal(topic, partition) else getReplicaOrException(topic, partition) @@ -395,7 +396,9 @@ class ReplicaManager(val config: KafkaConfig, else None - // read on log + // read on log, note that here we do not record the fetched message count and size + // since it may be re-read in the future; instead we should only record these metrics + // when the responses are sent val logReadInfo = localReplica.log match { case Some(log) => log.read(offset, fetchSize, maxOffsetOpt) @@ -404,9 +407,6 @@ class ReplicaManager(val config: KafkaConfig, FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) } - BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(logReadInfo.messageSet.sizeInBytes) - LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, None) } catch { // NOTE: Failed fetch requests metric is not incremented for known exceptions since it diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 6efa96a..8ef92c8 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -58,12 +58,12 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { def isCompleted(): Boolean = completed.get() /* - * Process for expiring a timed out request + * Process for expiring a timed out operation */ def expire(): Unit = complete /** - * Process for completing a request + * Process for completing a operation */ def complete(): Unit } @@ -99,10 +99,11 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In /** * Check if the operation can be completed, if not watch it based on the given watch keys * - * Note that a delayed operation can be watched on multiple keys, and hence due to concurrency may be - * found completed when trying to watch it on some later keys. In this case the operation is still - * treated as completed and hence no longer watched although it is still in the watch lists of - * the earlier keys. Those already watched elements will be later purged by the expire reaper. + * Note that a delayed operation can be watched on multiple keys. It is possible that + * an operation is completed after it has been added to the watch list for some, but + * not all of the keys. In this case, the operation is considered completed and won't + * be added to the watch list of the remaining keys. The expiration reaper thread will + * remove this operation from any watcher list in which the operation exists. * * @param operation the delayed operation to be checked * @param watchKeys keys for bookkeeping the operation @@ -143,7 +144,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In if(watchers == null) 0 else - watchers.completeWatched() + watchers.tryCompleteWatched() } /* @@ -181,7 +182,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In } // traverse the list and try to complete some watched elements - def completeWatched(): Int = { + def tryCompleteWatched(): Int = { var completed = 0 synchronized { val iter = requests.iterator() @@ -191,7 +192,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In // another thread has completed this request, just remove it iter.remove() } else { - if(curr.tryComplete()) { + if(curr synchronized curr.tryComplete()) { iter.remove() completed += 1 } -- 1.7.12.4 From 910ba44dec12105b721b2c2237af348620af51ab Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 26 Aug 2014 14:52:48 -0700 Subject: [PATCH 22/33] fix reaper thread 1 --- .../src/main/scala/kafka/server/DelayedFetch.scala | 18 ++----- .../main/scala/kafka/server/DelayedProduce.scala | 12 +---- core/src/main/scala/kafka/server/KafkaApis.scala | 2 +- .../main/scala/kafka/server/ReplicaManager.scala | 1 + .../main/scala/kafka/server/RequestPurgatory.scala | 61 +++++++++++++--------- .../unit/kafka/server/RequestPurgatoryTest.scala | 12 ++--- .../unit/kafka/server/ServerShutdownTest.scala | 2 +- .../scala/unit/kafka/server/SimpleFetchTest.scala | 5 +- 8 files changed, 52 insertions(+), 61 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index c30cc0a..68bf028 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -80,11 +80,11 @@ class DelayedFetch(delayMs: Long, if (endOffset.offsetOnOlderSegment(fetchOffset)) { // Case C, this can happen when the new follower replica fetching on a truncated leader debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) - return super.tryComplete() + return forceComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { // Case C, this can happen when the follower replica is lagging too much debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata)) - return super.tryComplete() + return forceComplete() } else if (fetchOffset.precedes(endOffset)) { // we need take the partition fetch size as upper bound when accumulating the bytes accumulatedSize += math.min(endOffset.positionDiff(fetchOffset), fetchStatus.fetchInfo.fetchSize) @@ -93,29 +93,21 @@ class DelayedFetch(delayMs: Long, } catch { case utpe: UnknownTopicOrPartitionException => // Case A debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) - return super.tryComplete() + return forceComplete() case nle: NotLeaderForPartitionException => // Case B debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) - return super.tryComplete() + return forceComplete() } } // Case D if (accumulatedSize >= fetchMetadata.fetchMinBytes) - super.tryComplete() + forceComplete() else false } /** - * Upon expire, complete the fetch request and return - */ - override def expire() { - debug("Expire fetch %s and return whatever fetch data is available".format(fetchMetadata)) - complete() - } - - /** * Upon completion, read whatever data is available and pass to the complete callback */ override def complete() { diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index e48a45c..4075e01 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -109,23 +109,15 @@ class DelayedProduce(delayMs: Long, // check if each partition has satisfied either case A or case B if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) - super.tryComplete() + forceComplete() else false } /** - * Upon expire, complete the produce request and return - */ - override def expire() { - debug("Expire produce %s and return the error codes".format(produceMetadata)) - complete() - } - - /** * Upon completion, return the current response status along with the error code per partition */ - def complete() { + override def complete() { val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) callbackOnComplete(responseStatus) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 932492f..0cc6a97 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -213,7 +213,7 @@ class KafkaApis(val requestChannel: RequestChannel, } // record the bytes out metrics only when the response is being sent - BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(data.messages.sizeInBytes) + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesOutRate.mark(data.messages.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index d68bd7d..ee0ccc1 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -719,6 +719,7 @@ class ReplicaManager(val config: KafkaConfig, replicaFetcherManager.shutdown() fetchRequestPurgatory.shutdown() producerRequestPurgatory.shutdown() + scheduler.shutdown() checkpointHighWatermarks() info("Shut down completely") } diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 8ef92c8..93dc8f4 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -34,16 +34,17 @@ import com.yammer.metrics.core.Gauge * a delayed fetch operation could be waiting for a given number of bytes to accumulate. */ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { - val completed = new AtomicBoolean(false) + private val completed = new AtomicBoolean(false) /* - * Check if the delayed operation can be completed by the caller + * Force completing the delayed operation, this function can be triggered when * - * Note that concurrent threads can check if an operation can be completed or not, - * but only the first thread will succeed in completing the operation and return - * true, others will still return false + * 1. The operation has been verified to be completable now + * 2. The operation has expired and hence need to be completed no matter what + * + * Return true iff the operation is completed by the caller */ - def tryComplete(): Boolean = { + def forceComplete(): Boolean = { if (completed.compareAndSet(false, true)) { complete() true @@ -57,15 +58,22 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { */ def isCompleted(): Boolean = completed.get() - /* - * Process for expiring a timed out operation - */ - def expire(): Unit = complete - /** - * Process for completing a operation + * Process for completing a operation; this function needs to be defined in subclasses */ def complete(): Unit + + /* + * Try to complete the delayed operation by first checking if the operation + * can be completed by now; and if yes execute the completion logic by calling forceComplete() + * + * Note that concurrent threads can check if an operation can be completed or not, + * but only the first thread will succeed in completing the operation and return + * true, others will still return false + * + * this function needs to be defined in subclasses + */ + def tryComplete(): Boolean } /** @@ -242,17 +250,22 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In } /** - * Get the next expired event + * Try to get the next expired event and force completing it */ - private def pollExpired(): T = { + private def expireNext(): Boolean = { while (true) { val curr = delayed.poll(200L, TimeUnit.MILLISECONDS) - if (curr == null.asInstanceOf[T]) - return null.asInstanceOf[T] - // try set the operation failed (and hence completed), if succeed return it; - // otherwise try to get the next expired operation since this one has been completed by others - if (curr.completed.compareAndSet(false, true)) { - return curr + if (curr != null.asInstanceOf[T]) { + // if the operation gets successfully completed, return; + // otherwise try to get the next expired operation since + // this one has been completed by others + if (curr synchronized curr.forceComplete()) { + debug("Expired delayed request %s and return the error codes".format(curr)) + return true + } + } else { + // if there are no expired operations yet, return + return false } } throw new RuntimeException("This should not happen") @@ -277,13 +290,9 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In purged } - override def doWork() { - // try to get the next expired operation and trigger its expiration process - val curr = pollExpired() - if (curr != null.asInstanceOf[T]) { - curr.expire() - } + // try to get the next expired operation and force completing it + expireNext() // see if we need to force a full purge if (size() >= purgeInterval) { debug("Beginning purgatory purge") diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala index a1d8927..27370bb 100644 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala @@ -70,8 +70,8 @@ class RequestPurgatoryTest extends JUnit3Suite { assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) r1.awaitExpiration() val elapsed = System.currentTimeMillis - start - assertTrue("r1 expired", r1.expired) - assertFalse("r2 hasn't expired", r2.expired) + assertTrue("r1 completed due to expiration", r1.isCompleted()) + assertFalse("r2 hasn't completed", r2.isCompleted()) assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) } @@ -123,7 +123,6 @@ class RequestPurgatoryTest extends JUnit3Suite { // A mock delayed request that can be completed / expired at will class MockDelayedRequest(delayMs: Long) extends DelayedRequest(delayMs) { var completable = false - var expired = false def awaitExpiration() { synchronized { @@ -134,19 +133,16 @@ class RequestPurgatoryTest extends JUnit3Suite { override def tryComplete() = { if (completable) - completed.compareAndSet(false, true) + forceComplete() else false } - override def expire() { + override def complete() { synchronized { - expired = true notify() } } - - override def complete() { /* do nothing */ } } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 3804a11..1bfb501 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -106,7 +106,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val newProps = TestUtils.createBrokerConfig(0, port) newProps.setProperty("delete.topic.enable", "true") val newConfig = new KafkaConfig(newProps) - var server = new KafkaServer(newConfig) + val server = new KafkaServer(newConfig) server.startup() server.shutdown() server.awaitShutdown() diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index f161f33..9bf0932 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -64,7 +64,6 @@ class SimpleFetchTest extends JUnit3Suite { // create nice mock since we don't particularly care about zkclient calls val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) - //EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) EasyMock.replay(zkClient) // create nice mock since we don't particularly care about scheduler calls @@ -74,7 +73,6 @@ class SimpleFetchTest extends JUnit3Suite { // create the log which takes read with either HW max offset or none max offset val log = EasyMock.createMock(classOf[Log]) EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes() - //EasyMock.expect(log) EasyMock.expect(log.read(0, fetchSize, Some(partitionHW))).andReturn( new FetchDataInfo( new LogOffsetMetadata(0L, 0L, 0), @@ -90,6 +88,7 @@ class SimpleFetchTest extends JUnit3Suite { // create the log manager that is aware of this mock log val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) EasyMock.expect(logManager.getLog(topicAndPartition)).andReturn(Some(log)).anyTimes() + EasyMock.expect(logManager.deleteLog(topicAndPartition)).anyTimes() EasyMock.replay(logManager) // create the replica manager @@ -114,6 +113,8 @@ class SimpleFetchTest extends JUnit3Suite { } override def tearDown() { + replicaManager.stopReplica(topic, partitionId, true) + replicaManager.shutdown() super.tearDown() } -- 1.7.12.4 From 1cf0e68c9cd60a163b52985deb4c729d48200e47 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 26 Aug 2014 15:54:59 -0700 Subject: [PATCH 23/33] unit test passed --- core/src/main/scala/kafka/server/ReplicaManager.scala | 6 +++--- .../kafka/server/HighwatermarkPersistenceTest.scala | 7 +++++++ .../scala/unit/kafka/server/ISRExpirationTest.scala | 17 ++++++++++++++--- .../scala/unit/kafka/server/ReplicaManagerTest.scala | 6 ++++++ .../test/scala/unit/kafka/server/SimpleFetchTest.scala | 4 +--- 5 files changed, 31 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index ee0ccc1..ed93f4c 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -714,13 +714,13 @@ class ReplicaManager(val config: KafkaConfig, } } - def shutdown() { + def shutdown(checkpointHW: Boolean = true) { info("Shutting down") replicaFetcherManager.shutdown() fetchRequestPurgatory.shutdown() producerRequestPurgatory.shutdown() - scheduler.shutdown() - checkpointHighWatermarks() + if (checkpointHW) + checkpointHighWatermarks() info("Shut down completely") } } diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 03a424d..8913fc1 100644 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -74,6 +74,9 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { fooPartition0Hw = hwmFor(replicaManager, topic, 0) assertEquals(leaderReplicaPartition0.highWatermark.messageOffset, fooPartition0Hw) EasyMock.verify(zkClient) + + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) } def testHighWatermarkPersistenceMultiplePartitions() { @@ -130,6 +133,10 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) assertEquals(10L, topic1Partition0Hw) EasyMock.verify(zkClient) + + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) + } def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = { diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index cd302aa..a703d27 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -36,8 +36,21 @@ class IsrExpirationTest extends JUnit3Suite { }) val topic = "foo" + val time = new MockTime + + var replicaManager: ReplicaManager = null + + override def setUp() { + super.setUp() + replicaManager = new ReplicaManager(configs.head, time, null, null, null, new AtomicBoolean(false)) + } + + override def tearDown() { + replicaManager.shutdown(false) + super.tearDown() + } + def testIsrExpirationForStuckFollowers() { - val time = new MockTime val log = getLogWithLogEndOffset(15L, 2) // set logEndOffset for leader to 15L // create one partition and all replicas @@ -61,7 +74,6 @@ class IsrExpirationTest extends JUnit3Suite { } def testIsrExpirationForSlowFollowers() { - val time = new MockTime // create leader replica val log = getLogWithLogEndOffset(15L, 1) // add one partition @@ -82,7 +94,6 @@ class IsrExpirationTest extends JUnit3Suite { private def getPartitionWithAllReplicasInIsr(topic: String, partitionId: Int, time: Time, config: KafkaConfig, localLog: Log): Partition = { val leaderId=config.brokerId - val replicaManager = new ReplicaManager(config, time, null, null, null, new AtomicBoolean(false)) val partition = replicaManager.getOrCreatePartition(topic, partitionId) val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index a9c4ddc..faa9071 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -42,6 +42,9 @@ class ReplicaManagerTest extends JUnit3Suite { val partition = rm.getOrCreatePartition(topic, 1) partition.getOrCreateReplica(1) rm.checkpointHighWatermarks() + + // shutdown the replica manager upon test completion + rm.shutdown(false) } @Test @@ -56,5 +59,8 @@ class ReplicaManagerTest extends JUnit3Suite { val partition = rm.getOrCreatePartition(topic, 1) partition.getOrCreateReplica(1) rm.checkpointHighWatermarks() + + // shutdown the replica manager upon test completion + rm.shutdown(false) } } diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 9bf0932..6473776 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -88,7 +88,6 @@ class SimpleFetchTest extends JUnit3Suite { // create the log manager that is aware of this mock log val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) EasyMock.expect(logManager.getLog(topicAndPartition)).andReturn(Some(log)).anyTimes() - EasyMock.expect(logManager.deleteLog(topicAndPartition)).anyTimes() EasyMock.replay(logManager) // create the replica manager @@ -113,8 +112,7 @@ class SimpleFetchTest extends JUnit3Suite { } override def tearDown() { - replicaManager.stopReplica(topic, partitionId, true) - replicaManager.shutdown() + replicaManager.shutdown(false) super.tearDown() } -- 1.7.12.4 From f5221d91569dd285230006c0662913196b256b9c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 27 Aug 2014 09:24:30 -0700 Subject: [PATCH 24/33] final pass: comments --- core/src/main/scala/kafka/cluster/Partition.scala | 9 +++++---- core/src/main/scala/kafka/log/Log.scala | 15 ++++++++++----- core/src/main/scala/kafka/server/DelayedProduce.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 12 +++++++----- core/src/main/scala/kafka/server/OffsetManager.scala | 10 ++++------ core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- .../src/main/scala/kafka/server/RequestPurgatory.scala | 4 ++-- .../test/scala/unit/kafka/server/SimpleFetchTest.scala | 18 +++++++++++------- 8 files changed, 41 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 7487a5e..1b475a8 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -231,7 +231,7 @@ class Partition(val topic: String, } /** - * Update the log end offset of a replica of the partition + * Update the log end offset of a certain replica of this partition */ def updateReplicaLEO(replicaId: Int, offset: LogOffsetMetadata) = { getReplica(replicaId) match { @@ -252,8 +252,9 @@ class Partition(val topic: String, } /** - * Check and maybe expand the ISR of the partition; - * this can happen when a replica's LEO has incremented + * Check and maybe expand the ISR of the partition. + * + * This function can be triggered when a replica's LEO has incremented */ def maybeExpandIsr(replicaId: Int) { inWriteLock(leaderIsrUpdateLock) { @@ -327,7 +328,7 @@ class Partition(val topic: String, /** * Check and maybe increment the high watermark of the partition; - * this can happen when + * this function can be triggered when * * 1. Partition ISR changed * 2. Any replica's LEO changed (e.g. leader LEO changed and the ISR is down to 1) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 6008135..dc2587d 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -31,6 +31,10 @@ import scala.collection.JavaConversions import com.yammer.metrics.core.Gauge +object LogAppendInfo { + val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, -1, -1, false) +} + /** * Struct to hold various quantities we compute about each message set before appending to the log * @param firstOffset The first offset in the message set @@ -40,13 +44,11 @@ import com.yammer.metrics.core.Gauge * @param codec The codec used in the message set * @param offsetsMonotonic Are the offsets in this message set monotonically increasing */ - -object LogAppendInfo { - val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, -1, -1, false) -} - case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) +/* + * Result metadata of a log append operation on the log + */ case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { def errorCode = error match { case None => ErrorMapping.NoError @@ -54,6 +56,9 @@ case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) } } +/* + * Result metadata of a log read operation on the log + */ case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Option[Throwable] = None) { def errorCode = error match { case None => ErrorMapping.NoError diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 4075e01..cc40c51 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -107,7 +107,7 @@ class DelayedProduce(delayMs: Long, } } - // check if each partition has satisfied either case A or case B + // check if each partition has satisfied at lease one of case A and case B if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) forceComplete() else diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 0cc6a97..e587a45 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -231,7 +231,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /** - * Service the offset request API + * Handle a offset request request */ def handleOffsetRequest(request: RequestChannel.Request) { val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] @@ -291,7 +291,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { + private def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { val segsArray = log.logSegments.toArray var offsetTimeArray: Array[(Long, Long)] = null if(segsArray.last.size > 0) @@ -364,7 +364,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /** - * Service the topic metadata request API + * Handle a topic metadata request */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] @@ -376,7 +376,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /* - * Service the Offset fetch API + * Handle an offset fetch request */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] @@ -396,7 +396,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /* - * Service the consumer metadata API + * Handle a consumer metadata request */ def handleConsumerMetadataRequest(request: RequestChannel.Request) { val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest] @@ -421,6 +421,8 @@ class KafkaApis(val requestChannel: RequestChannel, } def close() { + // TODO currently closing the API is an no-op since the API no longer maintain any modules + // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer debug("Shut down complete.") } } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index d7d060f..7b0e29d 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -35,16 +35,13 @@ import scala.collection._ import java.io.PrintStream import java.util.concurrent.atomic.AtomicBoolean import java.nio.ByteBuffer -import java.util.{Collections, Properties} +import java.util.Properties import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge import org.I0Itec.zkclient.ZkClient import kafka.api.ProducerResponseStatus -import collection.JavaConversions._ - - /** * Configuration settings for in-built offset management @@ -204,7 +201,7 @@ class OffsetManager(val config: OffsetManagerConfig, consumerId: String, generationId: Int, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], - responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + callbackOnComplete: immutable.Map[TopicAndPartition, Short] => Unit) { // first filter out partitions with offset metadata size exceeding limit // TODO: in the future we may want to only support atomic commit and hence fail the whole commit @@ -268,7 +265,8 @@ class OffsetManager(val config: OffsetManagerConfig, } } - responseCallback(commitStatus) + // finally trigger the callback logic passed from the API layer + callbackOnComplete(commitStatus) } // call replica manager to append the offset messages diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index ed93f4c..9be58da 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -396,7 +396,7 @@ class ReplicaManager(val config: KafkaConfig, else None - // read on log, note that here we do not record the fetched message count and size + // read on log; note that here we do not record the fetched message count and size // since it may be re-read in the future; instead we should only record these metrics // when the responses are sent val logReadInfo = localReplica.log match { diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 93dc8f4..809fa29 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -39,8 +39,8 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { /* * Force completing the delayed operation, this function can be triggered when * - * 1. The operation has been verified to be completable now - * 2. The operation has expired and hence need to be completed no matter what + * 1. The operation has been verified to be completable inside tryComplete() + * 2. The operation has expired and hence need to be completed right now * * Return true iff the operation is completed by the caller */ diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 6473776..eb59459 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -117,14 +117,18 @@ class SimpleFetchTest extends JUnit3Suite { } /** - * The scenario for this test is that there is one topic, "test-topic", one broker "0" that has - * one partition with one follower replica on broker "1". The leader replica on "0" - * has HW of "5" and LEO of "20". The follower on broker "1" has a local replica - * with a HW matching the leader's ("5") and LEO of "15", meaning it's not in-sync - * but is still in ISR (hasn't yet expired from ISR). + * The scenario for this test is that there is one topic that has one partition + * with one leader replica on broker "0" and one follower replica on broker "1" + * inside the replica manager's metadata. * - * When a normal consumer fetches data, it should only see data up to the HW of the leader, - * in this case up an offset of "5". + * The leader replica on "0" has HW of "5" and LEO of "20". The follower on + * broker "1" has a local replica with a HW matching the leader's ("5") and + * LEO of "15", meaning it's not in-sync but is still in ISR (hasn't yet expired from ISR). + * + * When a fetch operation with read committed data turned on is received, the replica manager + * should only return data up to the HW of the partition; when a fetch operation with read + * committed data turned off is received, the replica manager could return data up to the LEO + * of the local leader replica's log. */ def testReadFromLog() { -- 1.7.12.4 From 685c68fe0998f58dca14a0c1b449d90a96141d41 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 5 Sep 2014 14:07:29 -0700 Subject: [PATCH 25/33] rebase on KAFKA-1616 --- core/src/main/scala/kafka/cluster/Partition.scala | 4 +- .../src/main/scala/kafka/server/DelayedFetch.scala | 4 +- core/src/main/scala/kafka/server/KafkaApis.scala | 10 +-- .../main/scala/kafka/server/ReplicaManager.scala | 4 +- .../main/scala/kafka/server/RequestPurgatory.scala | 78 +++++++++++----------- .../scala/unit/kafka/server/SimpleFetchTest.scala | 8 +-- 6 files changed, 54 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 1b475a8..6f4cf9b 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -279,7 +279,7 @@ class Partition(val topic: String, replicaManager.isrExpandRate.mark() } // check if the HW of the partition can now be incremented - // since the replica maybe now in the ISR and its LEO has just incremented + // since the replica maybe now be in the ISR and its LEO has just incremented maybeIncrementLeaderHW(leaderReplica) case None => // nothing to do if no longer leader @@ -331,7 +331,7 @@ class Partition(val topic: String, * this function can be triggered when * * 1. Partition ISR changed - * 2. Any replica's LEO changed (e.g. leader LEO changed and the ISR is down to 1) + * 2. Any replica's LEO changed * * Note There is no need to acquire the leaderIsrUpdate lock here * since all callers of this private API acquire that lock diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 68bf028..1653173 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -78,11 +78,11 @@ class DelayedFetch(delayMs: Long, replica.logEndOffset if (endOffset.offsetOnOlderSegment(fetchOffset)) { - // Case C, this can happen when the new follower replica fetching on a truncated leader + // Case C, this can happen when the new the fetch operation is on a truncated leader debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) return forceComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { - // Case C, this can happen when the follower replica is lagging too much + // Case C, this can happen when the fetch operation is lagging too much debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata)) return forceComplete() } else if (fetchOffset.precedes(endOffset)) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index e587a45..966a803 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -163,7 +163,7 @@ class KafkaApis(val requestChannel: RequestChannel, // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s" + debug("Produce request with correlation id %d from client %s on partition %s failed due to %s" .format(produceRequest.correlationId, produceRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) errorInResponse = true @@ -207,7 +207,7 @@ class KafkaApis(val requestChannel: RequestChannel, // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here if (data.error != ErrorMapping.NoError && data.error != ErrorMapping.UnknownCode) { - warn("Fetch request with correlation id %d from client %s on partition %s failed due to %s" + debug("Fetch request with correlation id %d from client %s on partition %s failed due to %s" .format(fetchRequest.correlationId, fetchRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(data.error))) } @@ -231,7 +231,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /** - * Handle a offset request request + * Handle an offset request */ def handleOffsetRequest(request: RequestChannel.Request) { val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] @@ -263,11 +263,11 @@ class KafkaApis(val requestChannel: RequestChannel, // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages // are typically transient and there is no value in logging the entire stack trace for the same case utpe: UnknownTopicOrPartitionException => - warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition, utpe.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case nle: NotLeaderForPartitionException => - warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition,nle.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case e: Throwable => diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 9be58da..bf16dcb 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -396,9 +396,7 @@ class ReplicaManager(val config: KafkaConfig, else None - // read on log; note that here we do not record the fetched message count and size - // since it may be re-read in the future; instead we should only record these metrics - // when the responses are sent + // read on log val logReadInfo = localReplica.log match { case Some(log) => log.read(offset, fetchSize, maxOffsetOpt) diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 809fa29..585e1b7 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -37,10 +37,11 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { private val completed = new AtomicBoolean(false) /* - * Force completing the delayed operation, this function can be triggered when + * Force completing the delayed operation, if not already completed. + * This function can be triggered when * * 1. The operation has been verified to be completable inside tryComplete() - * 2. The operation has expired and hence need to be completed right now + * 2. The operation has expired and hence needs to be completed right now * * Return true iff the operation is completed by the caller */ @@ -59,7 +60,8 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { def isCompleted(): Boolean = completed.get() /** - * Process for completing a operation; this function needs to be defined in subclasses + * Process for completing an operation; This function needs to be defined in subclasses + * and would be called exactly once in forceComplete() */ def complete(): Unit @@ -91,14 +93,14 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In newGauge( "PurgatorySize", new Gauge[Int] { - def value = size() + def value = watched() } ) newGauge( - "NumDelayedOperations", + "NumDelayedRequests", new Gauge[Int] { - def value = expirationReaper.enqueued + def value = delayed() } ) @@ -155,12 +157,17 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In watchers.tryCompleteWatched() } - /* - * Return the size of the purgatory, which is size of watch lists plus the size of the expire reaper. - * Since an operation may still be in the watch lists even when it has been completed, this number - * may be larger than the number of real operations watched + /** + * Return the total size of watch lists the purgatory. Since an operation may be watched + * on multiple lists, and some of its watched entries may still be in the watch lists + * even when it has been completed, this number may be larger than the number of real operations watched */ - protected def size() = watchersForKey.values.map(_.watched).sum + expirationReaper.enqueued + def watched() = watchersForKey.values.map(_.watched).sum + + /** + * Return the number of delayed operations in the expiry queue + */ + def delayed() = expirationReaper.delayed /* * Return the watch list of the given key @@ -235,50 +242,41 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In false) { /* The queue storing all delayed operations */ - private val delayed = new DelayQueue[T] + private val delayedQueue = new DelayQueue[T] /* * Return the number of delayed operations kept by the reaper */ - def enqueued = delayed.size() + def delayed() = delayedQueue.size() /* * Add a operation to be expired */ def enqueue(t: T) { - delayed.add(t) + delayedQueue.add(t) } /** * Try to get the next expired event and force completing it */ - private def expireNext(): Boolean = { - while (true) { - val curr = delayed.poll(200L, TimeUnit.MILLISECONDS) - if (curr != null.asInstanceOf[T]) { - // if the operation gets successfully completed, return; - // otherwise try to get the next expired operation since - // this one has been completed by others - if (curr synchronized curr.forceComplete()) { - debug("Expired delayed request %s and return the error codes".format(curr)) - return true - } - } else { - // if there are no expired operations yet, return - return false + private def expireNext() { + val curr = delayedQueue.poll(200L, TimeUnit.MILLISECONDS) + if (curr != null.asInstanceOf[T]) { + // if there is an expired operation, try to force complete it + if (curr synchronized curr.forceComplete()) { + debug("Force complete expired delayed operation %s".format(curr)) } } - throw new RuntimeException("This should not happen") } /** * Delete all satisfied events from the delay queue and the watcher lists */ - private def purgeSatisfied(): Int = { + private def purgeCompleted(): Int = { var purged = 0 // purge the delayed queue - val iter = delayed.iterator() + val iter = delayedQueue.iterator() while (iter.hasNext) { val curr = iter.next() if (curr.isCompleted()) { @@ -293,13 +291,17 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In override def doWork() { // try to get the next expired operation and force completing it expireNext() - // see if we need to force a full purge - if (size() >= purgeInterval) { - debug("Beginning purgatory purge") - val purged = purgeSatisfied() - debug("Purged %d operations from delay queue.".format(purged)) - val numPurgedFromWatchers = watchersForKey.values.map(_.purgeCompleted()).sum - debug("Purged %d operations from watch lists.".format(numPurgedFromWatchers)) + // see if we need to purge the watch lists + if (RequestPurgatory.this.watched() >= purgeInterval) { + debug("Begin purging watch lists") + val purged = watchersForKey.values.map(_.purgeCompleted()).sum + debug("Purged %d elements from watch lists.".format(purged)) + } + // see if we need to purge the delayed request queue + if (delayed() >= purgeInterval) { + debug("Begin purging delayed queue") + val purged = purgeCompleted() + debug("Purged %d operations from delayed queue.".format(purged)) } } } diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index eb59459..ccf5e2e 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -132,10 +132,10 @@ class SimpleFetchTest extends JUnit3Suite { */ def testReadFromLog() { - assertEquals("", messagesToHW, replicaManager.readFromLocalLog(true, true, fetchInfo) - .get(topicAndPartition).get.info.messageSet.head.message) + assertEquals("Reading committed data should return messages only up to high watermark", messagesToHW, + replicaManager.readFromLocalLog(true, true, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) - assertEquals("", messagesToLEO, replicaManager.readFromLocalLog(true, false, fetchInfo) - .get(topicAndPartition).get.info.messageSet.head.message) + assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO, + replicaManager.readFromLocalLog(true, false, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) } } -- 1.7.12.4 From b4d0f13c2eea811a13d1eac804d4b1b41bfd7be3 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 5 Sep 2014 14:43:47 -0700 Subject: [PATCH 26/33] fix request purgatory test --- core/src/main/scala/kafka/server/KafkaApis.scala | 4 +- .../main/scala/kafka/server/RequestPurgatory.scala | 2 +- .../unit/kafka/server/RequestPurgatoryTest.scala | 52 ++++++---------------- 3 files changed, 17 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 966a803..234f2a3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -131,7 +131,7 @@ class KafkaApis(val requestChannel: RequestChannel, // Here we only print warnings for known errors; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here if (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.UnknownCode) { - warn("Offset commit request with correlation id %d from client %s on partition %s failed due to %s" + debug("Offset commit request with correlation id %d from client %s on partition %s failed due to %s" .format(offsetCommitRequest.correlationId, offsetCommitRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(errorCode))) } @@ -271,7 +271,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition,nle.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case e: Throwable => - warn("Error while responding to offset request", e) + error("Error while responding to offset request", e) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) ) } }) diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 585e1b7..07f965e 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -61,7 +61,7 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { /** * Process for completing an operation; This function needs to be defined in subclasses - * and would be called exactly once in forceComplete() + * and will be called exactly once in forceComplete() */ def complete(): Unit diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala index 27370bb..f131c2f 100644 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala @@ -20,6 +20,7 @@ package kafka.server import org.junit.Test import org.scalatest.junit.JUnit3Suite import junit.framework.Assert._ +import kafka.utils.TestUtils class RequestPurgatoryTest extends JUnit3Suite { @@ -27,15 +28,7 @@ class RequestPurgatoryTest extends JUnit3Suite { override def setUp() { super.setUp() -<<<<<<< HEAD -<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala - purgatory = new MockRequestPurgatory(5) -======= - purgatory = new MockDelayedOperationPurgatory() ->>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala -======= - purgatory = new RequestPurgatory[MockDelayedRequest]() ->>>>>>> unit test fix 1 + purgatory = new RequestPurgatory[MockDelayedRequest](0, 5) } override def tearDown() { @@ -75,29 +68,29 @@ class RequestPurgatoryTest extends JUnit3Suite { assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) } -<<<<<<< HEAD @Test def testRequestPurge() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r12 = new DelayedRequest(Array("test1", "test2"), null, 100000L) - val r23 = new DelayedRequest(Array("test2", "test3"), null, 100000L) - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r12) - purgatory.checkAndMaybeWatch(r23) + val r1 = new MockDelayedRequest(100000L) + val r2 = new MockDelayedRequest(100000L) + purgatory.tryCompleteElseWatch(r1, Array("test1")) + purgatory.tryCompleteElseWatch(r2, Array("test1", "test2")) + purgatory.tryCompleteElseWatch(r1, Array("test2", "test3")) assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) - // satisfy one of the requests, it should then be purged from the watch list with purge interval 5 - r12.satisfied.set(true) + // complete one of the operations, it should + // eventually be purged from the watch list with purge interval 5 + r2.completable = true + r2.tryComplete() TestUtils.waitUntilTrue(() => purgatory.watched() == 3, - "Purgatory should have 3 watched elements instead of " + + purgatory.watched(), 1000L) + "Purgatory should have 3 watched elements instead of " + purgatory.watched(), 1000L) TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r1) + purgatory.tryCompleteElseWatch(r1, Array("test1")) + purgatory.tryCompleteElseWatch(r1, Array("test1")) TestUtils.waitUntilTrue(() => purgatory.watched() == 5, "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) @@ -105,29 +98,12 @@ class RequestPurgatoryTest extends JUnit3Suite { "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) } -<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala -<<<<<<< HEAD:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala - class MockRequestPurgatory(purge: Int) extends RequestPurgatory[DelayedRequest](purgeInterval = purge) { -======= - class MockDelayedOperationPurgatory extends DelayedOperationPurgatory[DelayedRequest] { ->>>>>>> step 1:core/src/test/scala/unit/kafka/server/DelayedOperationPurgatoryTest.scala -======= - class MockRequestPurgatory extends RequestPurgatory[DelayedRequest] { ->>>>>>> change name back:core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala - val satisfied = mutable.Set[DelayedRequest]() - val expired = mutable.Set[DelayedRequest]() - def awaitExpiration(delayed: DelayedRequest) = { - delayed synchronized { - delayed.wait() -======= - // A mock delayed request that can be completed / expired at will class MockDelayedRequest(delayMs: Long) extends DelayedRequest(delayMs) { var completable = false def awaitExpiration() { synchronized { wait() ->>>>>>> unit test fix 1 } } -- 1.7.12.4 From 3c4afac6c608d81953e77b1e45bc6e7dd36ffbd1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 13 Oct 2014 14:54:08 -0700 Subject: [PATCH 27/33] dummy --- core/src/main/scala/kafka/cluster/Partition.scala | 3 ++- core/src/main/scala/kafka/server/OffsetManager.scala | 2 ++ core/src/main/scala/kafka/server/ReplicaManager.scala | 7 ++++--- .../kafka/api/ProducerFailureHandlingTest.scala | 18 +++++++----------- .../scala/unit/kafka/producer/SyncProducerTest.scala | 4 ++-- 5 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 6f4cf9b..25e95ef 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -301,6 +301,7 @@ class Partition(val topic: String, val minIsr = leaderReplica.log.get.config.minInSyncReplicas trace("%d/%d acks satisfied for %s-%d".format(numAcks, requiredAcks, topic, partitionId)) + if (requiredAcks < 0 && leaderReplica.highWatermark.messageOffset >= requiredOffset ) { /* * requiredAcks < 0 means acknowledge after all replicas in ISR @@ -397,7 +398,7 @@ class Partition(val topic: String, stuckReplicas ++ slowReplicas } - def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int=0) = { + def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = { inReadLock(leaderIsrUpdateLock) { val leaderReplicaOpt = leaderReplicaIfLocal() leaderReplicaOpt match { diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 7b0e29d..66197a9 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -145,6 +145,8 @@ class OffsetManager(val config: OffsetManagerConfig, 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 } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index bf16dcb..94431c3 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -234,7 +234,7 @@ class ReplicaManager(val config: KafkaConfig, responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { val sTime = SystemTime.milliseconds - val localProduceResults = appendToLocalLog(messagesPerPartition) + val localProduceResults = appendToLocalLog(messagesPerPartition, requiredAcks) debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) val produceStatus = localProduceResults.map{ case (topicAndPartition, result) => @@ -273,14 +273,15 @@ class ReplicaManager(val config: KafkaConfig, /** * Append the messages to the local replica logs */ - private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet]): Map[TopicAndPartition, LogAppendResult] = { + private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet], + requiredAcks: Short): Map[TopicAndPartition, LogAppendResult] = { trace("Append [%s] to local log ".format(messagesPerPartition)) messagesPerPartition.map { case (topicAndPartition, messages) => try { val partitionOpt = getPartition(topicAndPartition.topic, topicAndPartition.partition) val info = partitionOpt match { case Some(partition) => - partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet]) + partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks) case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" .format(topicAndPartition, localBrokerId)) } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 209a409..681adb3 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -305,13 +305,11 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testNotEnoughReplicas() { val topicName = "minisrtest" - val topicProps = new Properties(); - topicProps.put("min.insync.replicas","3"); - + val topicProps = new Properties() + topicProps.put("min.insync.replicas","3") TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) - val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) try { producer3.send(record).get @@ -327,18 +325,16 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testNotEnoughReplicasAfterBrokerShutdown() { val topicName = "minisrtest2" - val topicProps = new Properties(); - topicProps.put("min.insync.replicas","2"); - + val topicProps = new Properties() + topicProps.put("min.insync.replicas","2") TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) - val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) - // This should work + // this should work with all brokers up and running producer3.send(record).get - //shut down one broker + // shut down one broker servers.head.shutdown() servers.head.awaitShutdown() try { @@ -351,8 +347,8 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes } } + // restart the server for cleanly tear down servers.head.startup() - } private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index fb61d55..d60d8e0 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -237,8 +237,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { props.put("request.required.acks", "-1") val producer = new SyncProducer(new SyncProducerConfig(props)) - val topicProps = new Properties(); - topicProps.put("min.insync.replicas","2"); + val topicProps = new Properties() + topicProps.put("min.insync.replicas","2") AdminUtils.createTopic(zkClient, topicName, 1, 1,topicProps) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicName, 0) -- 1.7.12.4 From fdb166979e384e6218081d9d6274a2189baad33e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 13 Oct 2014 19:39:12 -0700 Subject: [PATCH 28/33] last pass --- core/src/main/scala/kafka/server/DelayedProduce.scala | 4 ---- core/src/main/scala/kafka/server/KafkaApis.scala | 6 +++--- core/src/main/scala/kafka/server/RequestPurgatory.scala | 2 +- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index cc40c51..1c8fad4 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -65,10 +65,6 @@ class DelayedProduce(delayMs: Long, trace("Initial partition status for %s is %s".format(topicAndPartition, status)) } - produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => - trace("Debugging: initial partition status for %s is %s".format(topicAndPartition, status)) - } - /** * The delayed produce request can be completed if every partition * it produces to is satisfied by one of the following: diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 234f2a3..7b09dd9 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -128,7 +128,7 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending the response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { commitStatus.foreach { case (topicAndPartition, errorCode) => - // Here we only print warnings for known errors; if it is unknown, it will cause + // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here if (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.UnknownCode) { debug("Offset commit request with correlation id %d from client %s on partition %s failed due to %s" @@ -160,7 +160,7 @@ class KafkaApis(val requestChannel: RequestChannel, def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { var errorInResponse = false responseStatus.foreach { case (topicAndPartition, status) => - // Here we only print warnings for known errors; if it is unknown, it will cause + // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { debug("Produce request with correlation id %d from client %s on partition %s failed due to %s" @@ -204,7 +204,7 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending the response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { responsePartitionData.foreach { case (topicAndPartition, data) => - // Here we only print warnings for known errors; if it is unknown, it will cause + // we only print warnings for known errors here; if it is unknown, it will cause // an error message in the replica manager already and hence can be ignored here if (data.error != ErrorMapping.NoError && data.error != ErrorMapping.UnknownCode) { debug("Fetch request with correlation id %d from client %s on partition %s failed due to %s" diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 07f965e..3ac0037 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -250,7 +250,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In def delayed() = delayedQueue.size() /* - * Add a operation to be expired + * Add an operation to be expired */ def enqueue(t: T) { delayedQueue.add(t) -- 1.7.12.4 From c1f6bd9cff98396e62d544ada3212e36c598df4e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 16 Oct 2014 21:14:27 -0700 Subject: [PATCH 29/33] Incorporate Jun's comments after rebase --- core/src/main/scala/kafka/server/DelayedFetch.scala | 8 ++++---- core/src/main/scala/kafka/server/DelayedProduce.scala | 7 ++++--- core/src/main/scala/kafka/server/KafkaApis.scala | 5 +++-- core/src/main/scala/kafka/server/ReplicaManager.scala | 6 +++--- core/src/main/scala/kafka/server/RequestPurgatory.scala | 3 ++- 5 files changed, 16 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 1653173..0976f19 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -59,7 +59,7 @@ class DelayedFetch(delayMs: Long, * The request can be completed if: * * Case A: This broker is no longer the leader for some partitions it tries to fetch - * Case B: This broker is does not know of some partitions it tries to fetch + * Case B: This broker does not know of some partitions it tries to fetch * Case C: The fetch offset locates not on the last segment of the log * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes */ @@ -78,7 +78,7 @@ class DelayedFetch(delayMs: Long, replica.logEndOffset if (endOffset.offsetOnOlderSegment(fetchOffset)) { - // Case C, this can happen when the new the fetch operation is on a truncated leader + // Case C, this can happen when the new fetch operation is on a truncated leader debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) return forceComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { @@ -91,10 +91,10 @@ class DelayedFetch(delayMs: Long, } } } catch { - case utpe: UnknownTopicOrPartitionException => // Case A + case utpe: UnknownTopicOrPartitionException => // Case B debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) return forceComplete() - case nle: NotLeaderForPartitionException => // Case B + case nle: NotLeaderForPartitionException => // Case A debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) return forceComplete() } diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 1c8fad4..e88418d 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -69,10 +69,11 @@ class DelayedProduce(delayMs: Long, * The delayed produce request can be completed if every partition * it produces to is satisfied by one of the following: * - * Case A: This broker is no longer the leader: should return error + * Case A: This broker is no longer the leader: return error * Case B: This broker is the leader: - * B.1 - If there was a localError (when writing to the local log): should return error - * B.2 - else, at least requiredAcks replicas should be caught up to this request. + * B.1 - If there was a local error thrown while checking if at least requiredAcks + * replicas have caught up to this request: return error + * B.2 - Otherwise, return the response with no error. */ override def tryComplete(): Boolean = { // check for each partition if it still has pending acks diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 7b09dd9..3f062fa 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -128,8 +128,9 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending the response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { commitStatus.foreach { case (topicAndPartition, errorCode) => - // we only print warnings for known errors here; if it is unknown, it will cause - // an error message in the replica manager already and hence can be ignored here + // we only print warnings for known errors here; only replica manager could see an unknown + // exception while trying to write the offset message to the local log, and it will log + // an error message and write the error code in this case; hence it can be ignored here if (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.UnknownCode) { debug("Offset commit request with correlation id %d from client %s on partition %s failed due to %s" .format(offsetCommitRequest.correlationId, offsetCommitRequest.clientId, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 94431c3..b06e093 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -695,9 +695,8 @@ class ReplicaManager(val config: KafkaConfig, private def getLeaderPartitions() : List[Partition] = { allPartitions.values.filter(_.leaderReplicaIfLocal().isDefined).toList } - /** - * Flushes the highwatermark value for all partitions to the highwatermark file - */ + + // Flushes the highwatermark value for all partitions to the highwatermark file def checkpointHighWatermarks() { val replicas = allPartitions.values.map(_.getReplica(config.brokerId)).collect{case Some(replica) => replica} val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParentFile.getAbsolutePath) @@ -713,6 +712,7 @@ class ReplicaManager(val config: KafkaConfig, } } + // High watermark do not need to be checkpointed only when under uni tests def shutdown(checkpointHW: Boolean = true) { info("Shutting down") replicaFetcherManager.shutdown() diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 3ac0037..8da3140 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -67,7 +67,8 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { /* * Try to complete the delayed operation by first checking if the operation - * can be completed by now; and if yes execute the completion logic by calling forceComplete() + * can be completed by now. If yes execute the completion logic by calling + * forceComplete() and return true; otherwise return false * * Note that concurrent threads can check if an operation can be completed or not, * but only the first thread will succeed in completing the operation and return -- 1.7.12.4 From 4b05f9425091e3e3c7c7982289176d6ebbd429a1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 17 Oct 2014 09:54:54 -0700 Subject: [PATCH 30/33] Incorporate Jun's comments --- core/src/main/scala/kafka/server/DelayedFetch.scala | 2 +- core/src/main/scala/kafka/server/DelayedProduce.scala | 6 +++--- core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- core/src/main/scala/kafka/server/RequestPurgatory.scala | 13 +++++++++---- .../test/scala/unit/kafka/server/RequestPurgatoryTest.scala | 2 +- 5 files changed, 15 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 0976f19..a9cbaea 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -110,7 +110,7 @@ class DelayedFetch(delayMs: Long, /** * Upon completion, read whatever data is available and pass to the complete callback */ - override def complete() { + override def onComplete() { val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, fetchMetadata.fetchOnlyCommitted, fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index e88418d..28cc4b3 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -69,10 +69,10 @@ class DelayedProduce(delayMs: Long, * The delayed produce request can be completed if every partition * it produces to is satisfied by one of the following: * - * Case A: This broker is no longer the leader: return error + * Case A: This broker is no longer the leader: set an error in response * Case B: This broker is the leader: * B.1 - If there was a local error thrown while checking if at least requiredAcks - * replicas have caught up to this request: return error + * replicas have caught up to this request: set an error in response * B.2 - Otherwise, return the response with no error. */ override def tryComplete(): Boolean = { @@ -114,7 +114,7 @@ class DelayedProduce(delayMs: Long, /** * Upon completion, return the current response status along with the error code per partition */ - override def complete() { + override def onComplete() { val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) callbackOnComplete(responseStatus) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b06e093..b57cb88 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -712,7 +712,7 @@ class ReplicaManager(val config: KafkaConfig, } } - // High watermark do not need to be checkpointed only when under uni tests + // High watermark do not need to be checkpointed only when under unit tests def shutdown(checkpointHW: Boolean = true) { info("Shutting down") replicaFetcherManager.shutdown() diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 8da3140..1acefd6 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -32,6 +32,11 @@ import com.yammer.metrics.core.Gauge * An operation whose processing needs to be delayed for at most the given delayMs. For example * a delayed produce operation could be waiting for specified number of acks; or * a delayed fetch operation could be waiting for a given number of bytes to accumulate. + * + * The logic upon completing a delayed operation is defined in onComplete() which needs to be instantiated + * by the user; this logic can be triggered by either forceComplete(), which force calling onComplete() + * if no others have ever complete it, or tryComplete(), which first check if the operation can be completed + * or not now (this logic also needs to be instantiated by the user), and if yes call forceComplete(). */ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { private val completed = new AtomicBoolean(false) @@ -47,7 +52,7 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { */ def forceComplete(): Boolean = { if (completed.compareAndSet(false, true)) { - complete() + onComplete() true } else { false @@ -63,12 +68,12 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { * Process for completing an operation; This function needs to be defined in subclasses * and will be called exactly once in forceComplete() */ - def complete(): Unit + def onComplete(): Unit /* * Try to complete the delayed operation by first checking if the operation * can be completed by now. If yes execute the completion logic by calling - * forceComplete() and return true; otherwise return false + * forceComplete() and return true iff forceComplete returns true; otherwise return false * * Note that concurrent threads can check if an operation can be completed or not, * but only the first thread will succeed in completing the operation and return @@ -118,7 +123,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In * * @param operation the delayed operation to be checked * @param watchKeys keys for bookkeeping the operation - * @return true iff the delayed operations can be completed + * @return true iff the delayed operations can be completed by the caller */ def tryCompleteElseWatch(operation: T, watchKeys: Seq[Any]): Boolean = { for(key <- watchKeys) { diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala index f131c2f..a7720d5 100644 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala @@ -114,7 +114,7 @@ class RequestPurgatoryTest extends JUnit3Suite { false } - override def complete() { + override def onComplete() { synchronized { notify() } -- 1.7.12.4 From 2ee042eb60211fa48530645daf39455357d4d10f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 22 Oct 2014 18:52:36 -0700 Subject: [PATCH 31/33] Incorporate Joel's comments --- .../src/main/scala/kafka/api/ProducerRequest.scala | 4 ++ core/src/main/scala/kafka/cluster/Partition.scala | 4 +- core/src/main/scala/kafka/log/Log.scala | 19 -------- .../src/main/scala/kafka/server/DelayedFetch.scala | 9 ++-- .../main/scala/kafka/server/DelayedProduce.scala | 4 +- core/src/main/scala/kafka/server/KafkaApis.scala | 7 ++- .../main/scala/kafka/server/OffsetManager.scala | 53 ++++++++++++---------- .../main/scala/kafka/server/ReplicaManager.scala | 22 ++++++++- .../kafka/api/ProducerFailureHandlingTest.scala | 2 +- 9 files changed, 70 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index b062406..570b2da 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -152,5 +152,9 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, producerRequest.append("; TopicAndPartition: " + topicPartitionMessageSizeMap.mkString(",")) producerRequest.toString() } + + def emptyData(){ + data.clear() + } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 25e95ef..1be5700 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -242,8 +242,8 @@ class Partition(val topic: String, // if it is not in the ISR yet maybeExpandIsr(replicaId) - debug("Recorded replica %d LEO position %d for partition [%s,%d]." - .format(replicaId, offset.messageOffset, topic, partitionId)) + debug("Recorded replica %d log end offset (LEO) position %d for partition %s." + .format(replicaId, offset.messageOffset, TopicAndPartition(topic, partitionId))) case None => throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + " is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId, diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index dc2587d..37b4a85 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -46,25 +46,6 @@ object LogAppendInfo { */ case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) -/* - * Result metadata of a log append operation on the log - */ -case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { - def errorCode = error match { - case None => ErrorMapping.NoError - case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) - } -} - -/* - * Result metadata of a log read operation on the log - */ -case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Option[Throwable] = None) { - def errorCode = error match { - case None => ErrorMapping.NoError - case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) - } -} /** * An append-only log for storing messages. diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index a9cbaea..1ccbb4b 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -52,7 +52,7 @@ case class FetchMetadata(fetchMinBytes: Int, class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, - callbackOnComplete: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) extends DelayedRequest(delayMs) { /** @@ -62,6 +62,8 @@ class DelayedFetch(delayMs: Long, * Case B: This broker does not know of some partitions it tries to fetch * Case C: The fetch offset locates not on the last segment of the log * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes + * + * Upon completion, should return whatever data is available for each valid partition */ override def tryComplete() : Boolean = { var accumulatedSize = 0 @@ -82,7 +84,8 @@ class DelayedFetch(delayMs: Long, debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) return forceComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { - // Case C, this can happen when the fetch operation is lagging too much + // Case C, this can happen when the fetch operation is falling behind the current segment + // or the partition has just rolled a new segment debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata)) return forceComplete() } else if (fetchOffset.precedes(endOffset)) { @@ -118,6 +121,6 @@ class DelayedFetch(delayMs: Long, val fetchPartitionData = logReadResults.mapValues(result => FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) - callbackOnComplete(fetchPartitionData) + responseCallback(fetchPartitionData) } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 28cc4b3..79c6483 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -49,7 +49,7 @@ case class ProduceMetadata(produceRequiredAcks: Short, class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, - callbackOnComplete: Map[TopicAndPartition, ProducerResponseStatus] => Unit) + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) extends DelayedRequest(delayMs) { // first update the acks pending variable according to the error code @@ -116,7 +116,7 @@ class DelayedProduce(delayMs: Long, */ override def onComplete() { val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) - callbackOnComplete(responseStatus) + responseCallback(responseStatus) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 3f062fa..968b0c4 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -194,6 +194,11 @@ class KafkaApis(val requestChannel: RequestChannel, produceRequest.requiredAcks, produceRequest.data, sendResponseCallback) + + // if the request is put into the purgatory, it will have a held reference + // and hence cannot be garbage collected; hence we clear its data here in + // order to let GC re-claim its memory since it is already appended to log + produceRequest.emptyData() } /** @@ -222,7 +227,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) } - // call the replica manager to append messages to the replicas + // call the replica manager to fetch messages from the local replica replicaManager.fetchMessages( fetchRequest.maxWait.toLong, fetchRequest.replicaId, diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 66197a9..f810cb2 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -195,6 +195,13 @@ class OffsetManager(val config: OffsetManagerConfig, offsetsCache.put(key, offsetAndMetadata) } + /* + * Check if the offset metadata is valid + */ + def validateOffsetMetadata(metadata: String) : Boolean = { + metadata == null || metadata.length() <= config.maxMetadataSize + } + /** * Store offsets by appending it to the replicated log and then inserting to cache */ @@ -203,19 +210,12 @@ class OffsetManager(val config: OffsetManagerConfig, consumerId: String, generationId: Int, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], - callbackOnComplete: immutable.Map[TopicAndPartition, Short] => Unit) { + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { // first filter out partitions with offset metadata size exceeding limit // TODO: in the future we may want to only support atomic commit and hence fail the whole commit - var commitStatus = offsetMetadata.mapValues { offsetAndMetadata => - if (offsetAndMetadata.metadata != null && offsetAndMetadata.metadata.length() > config.maxMetadataSize) - ErrorMapping.OffsetMetadataTooLargeCode - else - ErrorMapping.NoError - } - val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - commitStatus.get(topicAndPartition).get == ErrorMapping.NoError + validateOffsetMetadata(offsetAndMetadata.metadata) } // construct the message set to append @@ -242,33 +242,36 @@ class OffsetManager(val config: OffsetManagerConfig, // the offset and metadata to cache iff the append status has no error val status = responseStatus(offsetTopicPartition) - if (status.error == ErrorMapping.NoError) { - filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(groupName, topicAndPartition), offsetAndMetadata) - } - } else { - debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" - .format(filteredOffsetMetadata, groupName, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) + val responseCode = + if (status.error == ErrorMapping.NoError) { + filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => + putOffset(GroupTopicPartition(groupName, 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, groupName, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) - // update the commit status error code with the corresponding log append error code - val commitErrorCode = + // 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 status.error - - commitStatus = commitStatus.mapValues { case errorCode => - if (errorCode == ErrorMapping.NoError) - commitErrorCode - else - errorCode } + + + // compute the final error codes for the commit response + val commitStatusView = offsetMetadata.mapValues { case offsetAndMetadata => + if (validateOffsetMetadata(offsetAndMetadata.metadata)) + responseCode + else + ErrorMapping.OffsetMetadataTooLargeCode } // finally trigger the callback logic passed from the API layer - callbackOnComplete(commitStatus) + responseCallback(commitStatusView) } // call replica manager to append the offset messages diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b57cb88..210020f 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -20,7 +20,7 @@ import kafka.api._ import kafka.common._ import kafka.utils._ import kafka.cluster.{Broker, Partition, Replica} -import kafka.log.{LogReadResult, LogAppendResult, LogAppendInfo, LogManager} +import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.common.TopicAndPartition @@ -39,6 +39,26 @@ import scala.Some import org.I0Itec.zkclient.ZkClient import com.yammer.metrics.core.Gauge +/* + * Result metadata of a log append operation on the log + */ +case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} + +/* + * Result metadata of a log read operation on the log + */ +case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} + object ReplicaManager { val HighWatermarkFilename = "replication-offset-checkpoint" } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 681adb3..8531f53 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -347,7 +347,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes } } - // restart the server for cleanly tear down + // restart the server servers.head.startup() } -- 1.7.12.4 From 3ce2a23d73b72f2e18b82e283a9512e4848f6321 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 28 Oct 2014 14:51:06 -0700 Subject: [PATCH 32/33] dummy --- core/src/main/scala/kafka/server/OffsetManager.scala | 8 ++++---- core/src/main/scala/kafka/server/ReplicaManager.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index f810cb2..95dac5e 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -196,9 +196,9 @@ class OffsetManager(val config: OffsetManagerConfig, } /* - * Check if the offset metadata is valid + * Check if the offset metadata length is valid */ - def validateOffsetMetadata(metadata: String) : Boolean = { + def validateOffsetMetadataLength(metadata: String) : Boolean = { metadata == null || metadata.length() <= config.maxMetadataSize } @@ -215,7 +215,7 @@ class OffsetManager(val config: OffsetManagerConfig, // first filter out partitions with offset metadata size exceeding limit // TODO: in the future we may want to only support atomic commit and hence fail the whole commit val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - validateOffsetMetadata(offsetAndMetadata.metadata) + validateOffsetMetadataLength(offsetAndMetadata.metadata) } // construct the message set to append @@ -264,7 +264,7 @@ class OffsetManager(val config: OffsetManagerConfig, // compute the final error codes for the commit response val commitStatusView = offsetMetadata.mapValues { case offsetAndMetadata => - if (validateOffsetMetadata(offsetAndMetadata.metadata)) + if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) responseCode else ErrorMapping.OffsetMetadataTooLargeCode diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 210020f..02fa382 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -121,7 +121,7 @@ class ReplicaManager(val config: KafkaConfig, * Try to complete some delayed produce requests with the request key; * this can be triggered when: * - * 1. The partition HW has changed (for acks = -1). + * 1. The partition HW has changed (for acks = -1) * 2. A follower replica's fetch operation is received (for acks > 1) */ def tryCompleteDelayedProduce(key: DelayedRequestKey) { @@ -133,7 +133,7 @@ class ReplicaManager(val config: KafkaConfig, * Try to complete some delayed fetch requests with the request key; * this can be triggered when: * - * 1. The partition HW has changed; + * 1. The partition HW has changed (for regular fetch) * 2. A new message set is appended to the local log (for follower fetch) */ def tryCompleteDelayedFetch(key: DelayedRequestKey) { -- 1.7.12.4 From 37f6db72b6c0f9d074fa892a054b69bfa30d7785 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 28 Oct 2014 15:06:34 -0700 Subject: [PATCH 33/33] dummy --- core/src/main/scala/kafka/server/DelayedProduce.scala | 2 +- core/src/main/scala/kafka/server/OffsetManager.scala | 8 ++++---- core/src/main/scala/kafka/server/RequestPurgatory.scala | 11 +++++++---- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 79c6483..8049e07 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -73,7 +73,7 @@ class DelayedProduce(delayMs: Long, * Case B: This broker is the leader: * B.1 - If there was a local error thrown while checking if at least requiredAcks * replicas have caught up to this request: set an error in response - * B.2 - Otherwise, return the response with no error. + * B.2 - Otherwise, set the response with no error. */ override def tryComplete(): Boolean = { // check for each partition if it still has pending acks diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 95dac5e..2957bc4 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -263,15 +263,15 @@ class OffsetManager(val config: OffsetManagerConfig, // compute the final error codes for the commit response - val commitStatusView = offsetMetadata.mapValues { case offsetAndMetadata => + val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) - responseCode + (topicAndPartition, responseCode) else - ErrorMapping.OffsetMetadataTooLargeCode + (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) } // finally trigger the callback logic passed from the API layer - responseCallback(commitStatusView) + responseCallback(commitStatus) } // call replica manager to append the offset messages diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 1acefd6..323b12e 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -33,10 +33,13 @@ import com.yammer.metrics.core.Gauge * a delayed produce operation could be waiting for specified number of acks; or * a delayed fetch operation could be waiting for a given number of bytes to accumulate. * - * The logic upon completing a delayed operation is defined in onComplete() which needs to be instantiated - * by the user; this logic can be triggered by either forceComplete(), which force calling onComplete() - * if no others have ever complete it, or tryComplete(), which first check if the operation can be completed - * or not now (this logic also needs to be instantiated by the user), and if yes call forceComplete(). + * The logic upon completing a delayed operation is defined in onComplete() and will be called exactly once. + * Once an operation is completed, isCompleted() will return true. onComplete() can be triggered by either + * forceComplete(), which forces calling onComplete() after delayMs if the operation is not yet completed, + * or tryComplete(), which first checks if the operation can be completed or not now, and if yes calls + * forceComplete(). + * + * A subclass of DelayedRequest needs to provide an implementation of both onComplete() and tryComplete(). */ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { private val completed = new AtomicBoolean(false) -- 1.7.12.4