kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Dong Lin (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (KAFKA-6194) Server crash while deleting segments
Date Fri, 10 Nov 2017 02:05:00 GMT

    [ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16246902#comment-16246902
] 

Dong Lin edited comment on KAFKA-6194 at 11/10/17 2:04 AM:
-----------------------------------------------------------

[~ijuma] Yes, there is behavior change in the sense that file deletion failure does not cause
broker shutdown but it does now. The current behavior seems to be the right behavior -- if
file deletion fails, we don't know what else can go wrong and it can be risky to keep the
broker running in general.

I think the right thing is to understand why the file is not deleted. If it can indeed happen
without affecting other logic that touches disk IO, we can update the code to avoid broker
shutdown if file deletion failed.


was (Author: lindong):
[~ijuma] Yes, there is behavior change in the sense that file deletion does not cause broker
shutdown but it does now. The current behavior seems to be the right behavior -- if file deletion
fails, we don't know what else can go wrong and it can be risky to keep the broker running
in general.

I think the right thing is to understand why the file is not deleted. If it can indeed happen
without affecting other logic that touches disk IO, we can update the code to avoid broker
shutdown if file deletion failed.

> Server crash while deleting segments
> ------------------------------------
>
>                 Key: KAFKA-6194
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6194
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 1.0.0
>         Environment: kafka version: 1.0
> client versions: 0.8.2.1-0.10.2.1
> platform: aws (eu-west-1a)
> nodes: 36 x r4.xlarge
> disk storage: 2.5 tb per node (~73% usage per node)
> topics: 250
> number of partitions: 48k (approx)
> os: ubuntu 14.04
> jvm: Java(TM) SE Runtime Environment (build 1.8.0_131-b11), Java HotSpot(TM) 64-Bit Server
VM (build 25.131-b11, mixed mode)
>            Reporter: Ben Corlett
>              Labels: regression
>
> We upgraded our R+D cluster to 1.0 in the hope that it would fix the deadlock from 0.11.0.*.
Sadly our cluster has the memory leak issues with 1.0 most likely from https://issues.apache.org/jira/browse/KAFKA-6185.
We are running one server on a patched version of 1.0 with the pull request from that.
> However today we have had two different servers fall over for non-heap related reasons.
The exceptions in the kafka log are :
> {code}
> [2017-11-09 15:32:04,037] ERROR Error while deleting segments for xxxxxxxxxx-49 in dir
/mnt/secure/kafka/datalog (kafka.server.LogDirFailureChannel)
> java.io.IOException: Delete of log 00000000000000000000.log.deleted failed.
>         at kafka.log.LogSegment.delete(LogSegment.scala:496)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$3(Log.scala:1596)
>         at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12)
>         at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
>         at kafka.log.Log.deleteSeg$1(Log.scala:1596)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$4(Log.scala:1599)
>         at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110)
>         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:61)
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:748)
> [2017-11-09 15:32:04,040] INFO [ReplicaManager broker=122] Stopping serving replicas
in dir /mnt/secure/kafka/datalog (kafka.server.ReplicaManager)
> [2017-11-09 15:32:04,041] ERROR Uncaught exception in scheduled task 'delete-file' (kafka.utils.KafkaScheduler)
> org.apache.kafka.common.errors.KafkaStorageException: Error while deleting segments for
xxxxxxxxxxxxxx-49 in dir /mnt/secure/kafka/datalog
> Caused by: java.io.IOException: Delete of log 00000000000000000000.log.deleted failed.
>         at kafka.log.LogSegment.delete(LogSegment.scala:496)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$3(Log.scala:1596)
>         at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12)
>         at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
>         at kafka.log.Log.deleteSeg$1(Log.scala:1596)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$4(Log.scala:1599)
>         at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110)
>         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:61)
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:748)
> .....
> [2017-11-09 15:32:05,341] ERROR Error while processing data for partition xxxxxxx-83
(kafka.server.ReplicaFetcherThread)
> org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an offline log
directory for partition xxxxxxx-83
> [2017-11-09 15:32:05,341] ERROR Error while processing data for partition xxxxxxx-89
(kafka.server.ReplicaFetcherThread)
> org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an offline log
directory for partition xxxxxxx-89
> [2017-11-09 15:32:05,341] ERROR Error while processing data for partition xxxxxxx-76
(kafka.server.ReplicaFetcherThread)
> .....
> 2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording the replica
LEO, the partition xxxxxxx-27 hasn't been created. (kafka.server.ReplicaManager)
> [2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording the replica
LEO, the partition xxxxxxxxx-79 hasn't been created. (kafka.server.ReplicaManager)
> [2017-11-09 15:32:05,622] FATAL Shutdown broker because all log dirs in /mnt/secure/kafka/datalog
have failed (kafka.log.LogManager)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message