From jira-return-9190-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Mon Jan 29 11:53:29 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id AC782180654 for ; Mon, 29 Jan 2018 11:53:29 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 9CE54160C53; Mon, 29 Jan 2018 10:53:29 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 9749A160C31 for ; Mon, 29 Jan 2018 11:53:28 +0100 (CET) Received: (qmail 28012 invoked by uid 500); 29 Jan 2018 10:53:27 -0000 Mailing-List: contact jira-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@kafka.apache.org Delivered-To: mailing list jira@kafka.apache.org Received: (qmail 27989 invoked by uid 99); 29 Jan 2018 10:53:27 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 29 Jan 2018 10:53:27 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 5192618040C for ; Mon, 29 Jan 2018 10:53:27 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.511 X-Spam-Level: X-Spam-Status: No, score=-109.511 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id wRtfbQ8wzSvs for ; Mon, 29 Jan 2018 10:53:24 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id CC2E461330 for ; Mon, 29 Jan 2018 10:52:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 09931E0117 for ; Mon, 29 Jan 2018 10:52:01 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 48DFD21301 for ; Mon, 29 Jan 2018 10:52:00 +0000 (UTC) Date: Mon, 29 Jan 2018 10:52:00 +0000 (UTC) From: "Dmitriy Matveev (JIRA)" To: jira@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (KAFKA-3410) Unclean leader election and "Halting because log truncation is not allowed" MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/KAFKA-3410?page=3Dcom.atlassian= .jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D1634= 3209#comment-16343209 ]=20 Dmitriy Matveev commented on KAFKA-3410: ---------------------------------------- We have faced the same problem. it's happened like this,=C2=A0all action i did was in Cloudera Manager: - 3 days before(it was friday) i did shrink zookeeper cluster and removed t= wo nodes. restarted whole cluster of zookeeper. - kafka stay in state, something like this, - "Outdate configuration need t= o restart" - today morning(monday) kafka run into trouble - two nodes lost connection = to zookeeper, and some thing went wrong. Logs one of the nodes: {code:java} 2018-01-29 03:37:17,480 INFO org.apache.zookeeper.ClientCnxn: Client sessio= n timed out, have not heard from server in 22837ms for sessionid 0x3612d6a5= 73501c1, closing socket connection and attempting reco n nect=20 2018-01-29 03:37:17,480 INFO org.apache.zookeeper.ClientCnxn: Client sessio= n timed out, have not heard from server in 27981ms for sessionid 0x7612d6a5= 62101dc, closing socket connection and attempting reco n nect=20 2018-01-29 03:37:17,480 INFO org.apache.zookeeper.ClientCnxn: Client sessio= n timed out, have not heard from server in 22836ms for sessionid 0x3612d6a5= 73501c2, closing socket connection and attempting reco n nect=20 2018-01-29 03:37:17,480 WARN kafka.server.ReplicaFetcherThread: [ReplicaFet= cherThread-11-337], Error in fetch kafka.server.ReplicaFetcherThread$FetchR= equest@1657a3a8. Possible cause: java.io.IOException:=20 C onnection to 337 was disconnected before the response was read 2018-01-29= 03:37:17,480 WARN kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread= -3-337], Error in fetch kafka.server.ReplicaFetcherThread$FetchRequest@34f5= 7a6c. Possible cause: java.io.IOException: C o nnection to 337 was disconnected before the response was read=20 .....................................=20 2018-01-29 03:37:17,580 INFO org.I0Itec.zkclient.ZkClient: zookeeper state = changed (Disconnected)=20 2018-01-29 03:37:17,580 INFO org.apache.curator.framework.state.ConnectionS= tateManager: State change: SUSPENDED 2018-01-29 03:37:17,580 INFO org.I0Ite= c.zkclient.ZkClient: zookeeper state changed (Disconnected)=20 2018-01-29 03:37:17,586 ERROR kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-15-337], Error for partition [tv20-long-retention-invalid-topic= ,23] to broker 337:org.apache.kafka.common.errors.NotLe aderForPartitionExc= eption: This server is not the leader for that topic-partition.=20 2018-01-29 03:37:17,587 ERROR kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-9-337], Error for partition [trckv20-ch-click,22] to broker 337= :org.apache.kafka.common.errors.NotLeaderForPartitionEx ception: This serve= r is not the leader for that topic-partition.=20 ................................. 2018-01-29 03:37:17,931 INFO org.apache.zookeeper.ClientCnxn: Opening socke= t connection to server zookeeper-02.cpp.com/192.168.1.101:2181. Will not at= tempt to authenticate using SASL (unknown error) 2018-01-29 03:37:17,931 IN= FO org.apache.zookeeper.ClientCnxn: Socket connection established to zookee= per-02.cpp.com/192.168.1.101:2181, initiating session=20 2018-01-29 03:37:17,933 INFO org.apache.zookeeper.ClientCnxn: Session estab= lishment complete on server zookeeper-02.cpp.com/192.168.1.101:2181, sessio= nid =3D 0x3612d6a573501c2, negotiated timeout =3D 30000 2018-01-29 03:37:17= ,934 INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed (SyncConnec= ted)=20 2018-01-29 03:37:18,063 INFO org.apache.zookeeper.ClientCnxn: Opening socke= t connection to server hb03.cpp.com/192.168.0.73:2181. Will not attempt to = authenticate using SASL (unknown error) 2018-01-29 03:37:18,063 INFO org.ap= ache.zookeeper.ClientCnxn: Socket connection established to hb03.cpp.com/19= 2.168.0.73:2181, initiating session=20 2018-01-29 03:37:18,065 INFO org.apache.zookeeper.ClientCnxn: Unable to rec= onnect to ZooKeeper service, session 0x3612d6a573501c1 has expired, closing= socket connection 2018-01-29 03:37:18,065 INFO org.I0Itec.zkclient.ZkClien= t: zookeeper state changed (Expired)=20 2018-01-29 03:37:18,065 INFO org.apache.zookeeper.ZooKeeper: Initiating cli= ent connection, connectString=3Dhb03.cpp.com:2181,zookeeper-01.cpp.com:2181= ,zookeeper-02.cpp.com:2181,zookeeper-03.cpp.com:2181,zookeeper-04.cpp.com:2= 181 sessionTimeout=3D10000 watcher=3Dorg.I0Itec.zkclient.ZkClient@1364a89= =20 2018-01-29 03:37:18,066 INFO kafka.controller.KafkaController$SessionExpira= tionListener: [SessionExpirationListener on 338], ZK expired; shut down all= controller components and try to re-elect=20 2018-01-29 03:37:18,066 INFO org.apache.zookeeper.ClientCnxn: EventThread s= hut down=20 2018-01-29 03:37:18,066 INFO kafka.controller.PartitionStateMachine: [Parti= tion state machine on Controller 338]: Stopped partition state machine=20 2018-01-29 03:37:18,067 INFO kafka.controller.ReplicaStateMachine: [Replica= state machine on controller 338]: Stopped replica state machine=20 2018-01-29 03:37:18,067 INFO kafka.controller.KafkaController: [Controller = 338]: Broker 338 resigned as the controller=20 2018-01-29 03:37:18,067 INFO org.apache.zookeeper.ClientCnxn: Opening socke= t connection to server hb03.cpp.com/192.168.0.73:2181. Will not attempt to = authenticate using SASL (unknown error)=20 2018-01-29 03:37:18,090 INFO org.apache.zookeeper.ClientCnxn: Socket connec= tion established to hb03.cpp.com/192.168.0.73:2181, initiating session=20 2018-01-29 03:37:18,152 INFO org.apache.zookeeper.ClientCnxn: Session estab= lishment complete on server hb03.cpp.com/192.168.0.73:2181, sessionid =3D 0= x7612d6a562113a6, negotiated timeout =3D 10000=20 2018-01-29 03:37:18,152 INFO org.I0Itec.zkclient.ZkClient: zookeeper state = changed (SyncConnected)=20 2018-01-29 03:37:18,153 INFO kafka.server.KafkaHealthcheck: re-registering = broker info in ZK for broker 338=20 2018-01-29 03:37:18,153 INFO kafka.utils.ZKCheckedEphemeral: Creating /brok= ers/ids/338 (is it secure? false)=20 2018-01-29 03:37:18,162 INFO org.apache.zookeeper.ClientCnxn: Opening socke= t connection to server zookeeper-01.cpp.com/192.168.1.100:2181. Will not at= tempt to authenticate using SASL (unknown error)=20 2018-01-29 03:37:18,222 ERROR kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-6-337], Error for partition [tv20-long-retention-topic,18] to b= roker 337:org.apache.kafka.common.errors.NotLeaderForP artitionException: This server is not the leader for that topic-partition.= =20 ..........................=20 2018-01-29 03:37:18,260 INFO kafka.utils.ZKCheckedEphemeral: Result of znod= e creation is: OK=20 2018-01-29 03:37:18,260 INFO kafka.utils.ZkUtils: Registered broker 338 at = path /brokers/ids/338 with addresses: PLAINTEXT -> EndPoint(historical03.cp= p.com,9092,PLAINTEXT)=20 2018-01-29 03:37:18,260 INFO kafka.server.KafkaHealthcheck: done re-registe= ring broker=20 2018-01-29 03:37:18,260 INFO kafka.server.KafkaHealthcheck: Subscribing to = /brokers/topics path to watch for new topics=20 2018-01-29 03:37:18,260 ERROR kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-0-337], Error for partition [trckv20-lead-processing,53] to bro= ker 337:org.apache.kafka.common.errors.NotLeaderForPar t itionException: This server is not the leader for that topic-partition.= =20 .............and it's ends with ......=20 2018-01-29 03:38:13,597 INFO kafka.server.ReplicaFetcherManager: [ReplicaFe= tcherManager on broker 338] Removed fetcher for partitions=20 2018-01-29 03:38:13,661 INFO kafka.server.ReplicaFetcherManager: [ReplicaFe= tcherManager on broker 338] Added fetcher for partitions List()=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-8-339], Halting because log truncation is not allowed for topic= trckv20-ch-click, Current leader 339's latest offset=20 4494396139 is less than replica 338's latest offset 4494403425=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-12-339], Halting because log truncation is not allowed for topi= c trckv20-ch-click, Current leader 339's latest offset 4494474772 is less than replica 338's latest offset 4494481669=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-10-339], Halting because log truncation is not allowed for topi= c trckv20-ch-click, Current leader 339's latest offset 4494433300 is less than replica 338's latest offset 4494439854=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-2-339], Halting because log truncation is not allowed for topic= trckv20-ch-click, Current leader 339's latest offset=20 4494387381 is less than replica 338's latest offset 4494394866=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-6-339], Halting because log truncation is not allowed for topic= trckv20-ch-click, Current leader 339's latest offset=20 4494466871 is less than replica 338's latest offset 4494472899=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-4-339], Halting because log truncation is not allowed for topic= trckv20-ch-click, Current leader 339's latest offset=20 4494407851 is less than replica 338's latest offset 4494413095=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-14-339], Halting because log truncation is not allowed for topi= c __consumer_offsets, Current leader 339's latest offs et 168707094 is less than replica 338's latest offset 168707288=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-5-339], Halting because log truncation is not allowed for topic= tv20-long-retention-topic, Current leader 339's lates t offset 18615 is less than replica 338's latest offset 18616=20 2018-01-29 03:38:17,254 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFe= tcherThread-0-339], Halting because log truncation is not allowed for topic= trckv20-ch-click, Current leader 339's latest offset=20 4494445097 is less than replica 338's latest offset 4494450915{code} - only one node kept working - consumers sucked with this message =C2=A0 {code:java} msg=3D"Error: kafka server: Request was for a consumer group that is not co= ordinated by this broker =C2=A0 our consumer wrote on golang, and we use sarama-cluster goclient{code} =C2=A0 Return kafka to normal state helped `unclean.leader.election.enable=3Dtrue`= ,=C2=A0but i had make 3 restaring of kafka cluster - set `unclean.leader.election.enable=3Dtrue` and restart, wait until kafka= goes normal, check logs and dashboard in cloudera. - set `unclean.leader.election.enable=3Dfalse` and restart, wait untill kaf= ka goes normal, restarted consumers. but something went wrong with consumer= s.=20 They was could not obtain consumer group. And one of the nodes of kafka, i = think, stucked - 2 two nodes writing to log, but this one was silent, and t= his one was an "Active controller" the same massage in log of consumer: {code:java} "msg=3D"Error: kafka server: Request was for a consumer group that is not c= oordinated by this broker" {code} - I have restarted "silent" node, and after this all was back to normal sta= te, another node become a "Active controller" Sorry if this a big amount of logs. > Unclean leader election and "Halting because log truncation is not allowe= d" > -------------------------------------------------------------------------= -- > > Key: KAFKA-3410 > URL: https://issues.apache.org/jira/browse/KAFKA-3410 > Project: Kafka > Issue Type: Bug > Reporter: James Cheng > Priority: Major > Labels: reliability > > I ran into a scenario where one of my brokers would continually shutdown,= with the error message: > [2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting becau= se log truncation is not allowed for topic test, Current leader 1's latest = offset 0 is less than replica 2's latest offset 151 (kafka.server.ReplicaFe= tcherThread) > I managed to reproduce it with the following scenario: > 1. Start broker1, with unclean.leader.election.enable=3Dfalse > 2. Start broker2, with unclean.leader.election.enable=3Dfalse > 3. Create topic, single partition, with replication-factor 2. > 4. Write data to the topic. > 5. At this point, both brokers are in the ISR. Broker1 is the partition l= eader. > 6. Ctrl-Z on broker2. (Simulates a GC pause or a slow network) Broker2 ge= ts dropped out of ISR. Broker1 is still the leader. I can still write data = to the partition. > 7. Shutdown Broker1. Hard or controlled, doesn't matter. > 8. rm -rf the log directory of broker1. (This simulates a disk replacemen= t or full hardware replacement) > 9. Resume broker2. It attempts to connect to broker1, but doesn't succeed= because broker1 is down. At this point, the partition is offline. Can't wr= ite to it. > 10. Resume broker1. Broker1 resumes leadership of the topic. Broker2 atte= mpts to join ISR, and immediately halts with the error message: > [2016-02-25 00:29:39,236] FATAL [ReplicaFetcherThread-0-1], Halting becau= se log truncation is not allowed for topic test, Current leader 1's latest = offset 0 is less than replica 2's latest offset 151 (kafka.server.ReplicaFe= tcherThread) > I am able to recover by setting unclean.leader.election.enable=3Dtrue on = my brokers. > I'm trying to understand a couple things: > * In step 10, why is broker1 allowed to resume leadership even though it = has no data? > * In step 10, why is it necessary to stop the entire broker due to one pa= rtition that is in this state? Wouldn't it be possible for the broker to co= ntinue to serve traffic for all the other topics, and just mark this one as= unavailable? > * Would it make sense to allow an operator to manually specify which brok= er they want to become the new master? This would give me more control over= how much data loss I am willing to handle. In this case, I would want brok= er2 to become the new master. Or, is that possible and I just don't know ho= w to do it? -- This message was sent by Atlassian JIRA (v7.6.3#76005)