Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id DC9FD2D20 for ; Thu, 21 Apr 2011 19:11:47 +0000 (UTC) Received: (qmail 24567 invoked by uid 500); 21 Apr 2011 19:11:47 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 24534 invoked by uid 500); 21 Apr 2011 19:11:47 -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 24526 invoked by uid 99); 21 Apr 2011 19:11:47 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 21 Apr 2011 19:11:47 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.116] (HELO hel.zones.apache.org) (140.211.11.116) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 21 Apr 2011 19:11:44 +0000 Received: from hel.zones.apache.org (hel.zones.apache.org [140.211.11.116]) by hel.zones.apache.org (Postfix) with ESMTP id CC506AC372 for ; Thu, 21 Apr 2011 19:11:05 +0000 (UTC) Date: Thu, 21 Apr 2011 19:11:05 +0000 (UTC) From: "Prakash Khemani (JIRA)" To: issues@hbase.apache.org Message-ID: <949850785.74110.1303413065833.JavaMail.tomcat@hel.zones.apache.org> In-Reply-To: <1531568889.13172.1300492289511.JavaMail.tomcat@hel.zones.apache.org> Subject: [jira] [Commented] (HBASE-3674) Treat ChecksumException as we would a ParseException splitting logs; else we replay split on every restart MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/HBASE-3674?page=3Dcom.atlassian= .jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D1302= 2873#comment-13022873 ]=20 Prakash Khemani commented on HBASE-3674: ---------------------------------------- This change got overwritten when HBASE-1364 was integrated. The change has to be added in HLogSplitter in the method getNextLogLine static private Entry getNextLogLine(Reader in, Path path, boolean skipError= s) throws CorruptedLogFileException, IOException { try { return in.next(); } catch (EOFException eof) { // truncated files are expected if a RS crashes (see HBASE-2643) LOG.info("EOF from hlog " + path + ". continuing"); return null; } catch (IOException e) { // If the IOE resulted from bad file format, // then this problem is idempotent and retrying won't help if (e.getCause() instanceof ParseException) { LOG.warn("ParseException from hlog " + path + ". continuing"); return null; } It might also be necessary to add this change to getReader(...) method protected Reader getReader(FileSystem fs, FileStatus file, Configuration = conf, boolean skipErrors) throws IOException, CorruptedLogFileException { Path path =3D file.getPath(); long length =3D file.getLen(); Reader in; // Check for possibly empty file. With appends, currently Hadoop report= s a // zero length even if the file has been sync'd. Revisit if HDFS-376 or // HDFS-878 is committed. if (length <=3D 0) { LOG.warn("File " + path + " might be still open, length is 0"); } try { recoverFileLease(fs, path, conf); try { in =3D getReader(fs, path, conf); } catch (EOFException e) { if (length <=3D 0) { // TODO should we ignore an empty, not-last log file if skip.erro= rs // is false? Either way, the caller should decide what to do. E.g= . // ignore if this is the last log in sequence. // TODO is this scenario still possible if the log has been // recovered (i.e. closed) LOG.warn("Could not open " + path + " for reading. File is empty"= , e); return null; > Treat ChecksumException as we would a ParseException splitting logs; else= we replay split on every restart > -------------------------------------------------------------------------= --------------------------------- > > Key: HBASE-3674 > URL: https://issues.apache.org/jira/browse/HBASE-3674 > Project: HBase > Issue Type: Bug > Components: wal > Reporter: stack > Assignee: stack > Priority: Critical > Fix For: 0.90.2 > > Attachments: 3674-v2.txt, 3674.txt > > > In short, a ChecksumException will fail log processing for a server so we= skip out w/o archiving logs. On restart, we'll then reprocess the logs --= hit the checksumexception anew, usually -- and so on. > Here is the splitLog method (edited): > {code} > private List splitLog(final FileStatus[] logfiles) throws IOExcep= tion { > .... > outputSink.startWriterThreads(entryBuffers); > =20 > try { > int i =3D 0; > for (FileStatus log : logfiles) { > Path logPath =3D log.getPath(); > long logLength =3D log.getLen(); > splitSize +=3D logLength; > LOG.debug("Splitting hlog " + (i++ + 1) + " of " + logfiles.lengt= h > + ": " + logPath + ", length=3D" + logLength); > try { > recoverFileLease(fs, logPath, conf); > parseHLog(log, entryBuffers, fs, conf); > processedLogs.add(logPath); > } catch (EOFException eof) { > // truncated files are expected if a RS crashes (see HBASE-2643= ) > LOG.info("EOF from hlog " + logPath + ". Continuing"); > processedLogs.add(logPath); > } catch (FileNotFoundException fnfe) { > // A file may be missing if the region server was able to archi= ve it > // before shutting down. This means the edits were persisted al= ready > LOG.info("A log was missing " + logPath + > ", probably because it was moved by the" + > " now dead region server. Continuing"); > processedLogs.add(logPath); > } catch (IOException e) { > // If the IOE resulted from bad file format, > // then this problem is idempotent and retrying won't help > if (e.getCause() instanceof ParseException || > e.getCause() instanceof ChecksumException) { > LOG.warn("ParseException from hlog " + logPath + ". continui= ng"); > processedLogs.add(logPath); > } else { > if (skipErrors) { > LOG.info("Got while parsing hlog " + logPath + > ". Marking as corrupted", e); > corruptedLogs.add(logPath); > } else { > throw e; > } > } > } > } > if (fs.listStatus(srcDir).length > processedLogs.size() > + corruptedLogs.size()) { > throw new OrphanHLogAfterSplitException( > "Discovered orphan hlog after split. Maybe the " > + "HRegionServer was not dead when we started"); > } > archiveLogs(srcDir, corruptedLogs, processedLogs, oldLogDir, fs, co= nf); =20 > } finally { > splits =3D outputSink.finishWritingAndClose(); > } > return splits; > } > {code} > Notice how we'll only archive logs only if we successfully split all logs= . We won't archive 31 of 35 files if we happen to get a checksum exception= on file 32. > I think we should treat a ChecksumException the same as a ParseException;= a retry will not fix it if HDFS could not get around the ChecksumException= (seems like in our case all replicas were corrupt). > Here is a play-by-play from the logs: > {code} > 813572 2011-03-18 20:31:44,687 DEBUG org.apache.hadoop.hbase.regionserver= .wal.HLogSplitter: Splitting hlog 34 of 35: hdfs://sv2borg170:9000/hbase/.l= ogs/sv2borg182,60020,1300384550664/sv2borg182%3A60020.1300461329481, length= =3D150 65662813573 2011-03-18 20:31:44,687 INFO org.apache.hadoop.hba= se.util.FSUtils: Recovering file hdfs://sv2borg170:9000/hbase/.logs/sv2borg= 182,60020,1300384550664/sv2borg182%3A60020.1300461329481 > .... > 813617 2011-03-18 20:31:46,238 INFO org.apache.hadoop.fs.FSInputChecker: = Found checksum error: b[0, 512]=3D000000cd000000502037383661376439656265643= 938636463343433386132343631323633303239371d6170695f6163636573735f746f6b656e= 5f7374 6174735f6275636b65740000000d9fa4d5dc0000012ec9c7cbaf00ffffffff= 000000010000006d0000005d000000080023376262626637646264316165613662346161306= 56334383436653732333132643a32390764656661756c746170695f616e64726f69645f6c6f= 67676564 696e5f73686172655f70656e64696e675f696e69740000012ec956b02804= 000000000000000100000000ffffffff4e128eca0eb078d0652b0abac467fd09000000cd000= 000502034663166613763666165333930666332653138346233393931303132623366331d61= 70695f6163 636573735f746f6b656e5f73746174735f6275636b65740000000d9fa4= d5dd0000012ec9c7cbaf00ffffffff000000010000006d0000005d000000080023663037343= 23966643036323862636530336238333938356239316237386633353a32390764656661756c= 746170695f61 6e64726f69645f6c6f67676564696e5f73686172655f70656e64696e= 675f696e69740000012ec9569f1804000000000000000100000000000000d30000004e20666= 63763393964303633343339666531666461633761616632613964643631331b6170695f6163= 636573735f746f 6b656e5f73746174735f68 > 813618 org.apache.hadoop.fs.ChecksumException: Checksum error: /blk_77817= 25413191608261:of:/hbase/.logs/sv2borg182,60020,1300384550664/sv2borg182%3A= 60020.1300461329481 at 15064576 > 813619 at org.apache.hadoop.fs.FSInputChecker.verifySum(FSInputCh= ecker.java:277) > 813620 at org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(F= SInputChecker.java:241) > 813621 at org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker= .java:176) > 813622 at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecke= r.java:193) > 813623 at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker= .java:158) > 813624 at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSCl= ient.java:1175) > 813625 at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuf= fer(DFSClient.java:1807) > 813626 at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DF= SClient.java:1859) > 813627 at java.io.DataInputStream.read(DataInputStream.java:132) > 813628 at java.io.DataInputStream.readFully(DataInputStream.java:= 178) > 813629 at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(Data= OutputBuffer.java:63) > 813630 at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputB= uffer.java:101) > 813631 at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceF= ile.java:1937) > 813632 at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceF= ile.java:1837) > 813633 at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceF= ile.java:1883) > 813634 at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLo= gReader.next(SequenceFileLogReader.java:198) > 813635 at org.apache.hadoop.hbase.regionserver.wal.SequenceFileLo= gReader.next(SequenceFileLogReader.java:172) > 813636 at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.p= arseHLog(HLogSplitter.java:429) > 813637 at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.s= plitLog(HLogSplitter.java:262) > 813638 at org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.s= plitLog(HLogSplitter.java:188) > 813639 at org.apache.hadoop.hbase.master.MasterFileSystem.splitLo= g(MasterFileSystem.java:197) > 813640 at org.apache.hadoop.hbase.master.MasterFileSystem.splitLo= gAfterStartup(MasterFileSystem.java:181) > 813641 at org.apache.hadoop.hbase.master.HMaster.finishInitializa= tion(HMaster.java:384) > 813642 at org.apache.hadoop.hbase.master.HMaster.run(HMaster.java= :283) > 813643 2011-03-18 20:31:46,239 WARN org.apache.hadoop.hdfs.DFSClient: Fou= nd Checksum error for blk_7781725413191608261_14589573 from 10.20.20.182:50= 010 at 15064576 > 813644 2011-03-18 20:31:46,240 INFO org.apache.hadoop.hdfs.DFSClient: Cou= ld not obtain block blk_7781725413191608261_14589573 from any node: java.io= .IOException: No live nodes contain current block. Will get new block locat= ions from namenode and retry... > 813645 2011-03-18 20:31:49,243 DEBUG org.apache.hadoop.hbase.regionserver= .wal.HLogSplitter: Pushed=3D80624 entries from hdfs://sv2borg170:9000/hbase= /.logs/sv2borg182,60020,1300384550664/sv2borg182%3A60020.1300461329481 > .... > {code} > See code above. On exception we'll dump edits read so far from this bloc= k, close out all writers tying off recovered.edits so far written. We'll s= kip archiving these files because we only archive if all files are processe= d; we won't archive files 30 of 35 if we failed splitting on file 31. > I think checksumexception should be treated same as a ParseException > =20 -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira