You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Julio Ng (JIRA)" <ji...@apache.org> on 2018/03/13 21:51:00 UTC
[jira] [Created] (KAFKA-6649) ReplicaFetcher stopped after non
fatal exception is thrown
Julio Ng created KAFKA-6649:
-------------------------------
Summary: ReplicaFetcher stopped after non fatal exception is thrown
Key: KAFKA-6649
URL: https://issues.apache.org/jira/browse/KAFKA-6649
Project: Kafka
Issue Type: Bug
Components: replication
Affects Versions: 1.1.0
Reporter: Julio Ng
We have seen several under-replication partitions, usually triggered by topic creation. After digging in the logs, we see the below:
{noformat}
[2018-03-12 22:40:17,641] ERROR [ReplicaFetcher replicaId=12, leaderId=0, fetcherId=1] Error due to (kafka.server.ReplicaFetcherThread)
kafka.common.KafkaException: Error processing data for partition [[TOPIC_NAME_REMOVED]]-84 offset 2098535
at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:204)
at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:169)
at scala.Option.foreach(Option.scala:257)
at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:169)
at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:166)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply$mcV$sp(AbstractFetcherThread.scala:166)
at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:166)
at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:166)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:250)
at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:164)
at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:111)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
Caused by: org.apache.kafka.common.errors.OffsetOutOfRangeException: Cannot increment the log start offset to 2098535 of partition [[TOPIC_NAME_REMOVED]]-84 since it is larger than the high watermark -1
[2018-03-12 22:40:17,641] INFO [ReplicaFetcher replicaId=12, leaderId=0, fetcherId=1] Stopped (kafka.server.ReplicaFetcherThread){noformat}
It looks like that after the ReplicaFetcherThread is stopped, the replicas start to lag behind, presumably because we are not fetching from the leader anymore. Further examining, the ShutdownableThread.scala object:
{noformat}
override def run(): Unit = {
info("Starting")
try {
while (isRunning)
doWork()
} catch {
case e: FatalExitError =>
shutdownInitiated.countDown()
shutdownComplete.countDown()
info("Stopped")
Exit.exit(e.statusCode())
case e: Throwable =>
if (isRunning)
error("Error due to", e)
} finally {
shutdownComplete.countDown()
}
info("Stopped")
}{noformat}
For the Throwable (non-fatal) case, it just exits the while loop and the thread stops doing work. I am not sure whether this is the intended behavior of the ShutdownableThread, or the exception should be caught and we should keep calling doWork()
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)