Return-Path: Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: (qmail 45812 invoked from network); 21 Dec 2010 22:26:22 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 21 Dec 2010 22:26:22 -0000 Received: (qmail 44520 invoked by uid 500); 21 Dec 2010 22:26:22 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 44432 invoked by uid 500); 21 Dec 2010 22:26:22 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 44424 invoked by uid 99); 21 Dec 2010 22:26:22 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 21 Dec 2010 22:26:22 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.22] (HELO thor.apache.org) (140.211.11.22) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 21 Dec 2010 22:26:21 +0000 Received: from thor (localhost [127.0.0.1]) by thor.apache.org (8.13.8+Sun/8.13.8) with ESMTP id oBLMQ0Ks000340 for ; Tue, 21 Dec 2010 22:26:01 GMT Message-ID: <29954877.253071292970360924.JavaMail.jira@thor> Date: Tue, 21 Dec 2010 17:26:00 -0500 (EST) From: "Jean-Daniel Cryans (JIRA)" To: issues@hbase.apache.org Subject: [jira] Commented: (HBASE-3380) Master failover can split logs of live servers In-Reply-To: <25144773.246771292956801570.JavaMail.jira@thor> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-3380?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12973941#action_12973941 ] Jean-Daniel Cryans commented on HBASE-3380: ------------------------------------------- bq. should splitLog() do one additional sanity test to make sure the directory it is about to split doesn't have a corresponding node in ZK (under /hbase/rs)? Currently that class doesn't talk with ZK, and since it's called MasterFileSystem I think it would make that class less cohesive but we could pass the list from HMaster. I think this could be done on top of counting the number of RS to expect during failover. bq. So I propose my patch w/ change of defaults to 1500/4500. I don't mind the changes as it helps on cold starts, but I think that in the failover case calling regionServerTracker.getOnlineServers().size() would be simple and reliable. > Master failover can split logs of live servers > ---------------------------------------------- > > Key: HBASE-3380 > URL: https://issues.apache.org/jira/browse/HBASE-3380 > Project: HBase > Issue Type: Bug > Reporter: Jean-Daniel Cryans > Priority: Blocker > Fix For: 0.90.0 > > Attachments: HBASE-3380-v1.patch > > > The reason why TestMasterFailover fails is that when it does the master failover, the new master doesn't wait long enough for all region servers to checkin so it goes ahead and split logs... which doesn't work because of the way lease timeouts work: > {noformat} > 2010-12-21 07:30:36,977 DEBUG [Master:0;vesta.apache.org:33170] wal.HLogSplitter(256): Splitting hlog 1 of 1: > hdfs://localhost:49187/user/hudson/.logs/vesta.apache.org,38743,1292916616340/vesta.apache.org%3A38743.1292916617204, length=0 > 2010-12-21 07:30:36,977 DEBUG [WriterThread-1] wal.HLogSplitter$WriterThread(619): Writer thread Thread[WriterThread-1,5,main]: starting > 2010-12-21 07:30:36,977 DEBUG [WriterThread-2] wal.HLogSplitter$WriterThread(619): Writer thread Thread[WriterThread-2,5,main]: starting > 2010-12-21 07:30:36,977 INFO [Master:0;vesta.apache.org:33170] util.FSUtils(625): Recovering file > hdfs://localhost:49187/user/hudson/.logs/vesta.apache.org,38743,1292916616340/vesta.apache.org%3A38743.1292916617204 > 2010-12-21 07:30:36,979 WARN [IPC Server handler 8 on 49187] namenode.FSNamesystem(1122): DIR* NameSystem.startFile: > failed to create file /user/hudson/.logs/vesta.apache.org,38743,1292916616340/vesta.apache.org%3A38743.1292916617204 for > DFSClient_hb_m_vesta.apache.org:33170_1292916630791 on client 127.0.0.1, because this file is already being created by > DFSClient_hb_rs_vesta.apache.org,38743,1292916616340_1292916617166 on 127.0.0.1 > ... > 2010-12-21 07:33:44,332 WARN [Master:0;vesta.apache.org:33170] util.FSUtils(644): Waited 187354ms for lease recovery on > hdfs://localhost:49187/user/hudson/.logs/vesta.apache.org,38743,1292916616340/vesta.apache.org%3A38743.1292916617204: > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to create file > /user/hudson/.logs/vesta.apache.org,38743,1292916616340/vesta.apache.org%3A38743.1292916617204 > for DFSClient_hb_m_vesta.apache.org:33170_1292916630791 on client 127.0.0.1, because this file is already > being created by DFSClient_hb_rs_vesta.apache.org,38743,1292916616340_1292916617166 on 127.0.0.1 > {noformat} > I think that we should always check in ZK the number of live region servers before waiting for them to check in, this way we know how many we should expect during failover. There's also a case where we still want to timeout, since RS can die during that time, but we should wait a bit longer. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.