[ https://issues.apache.org/jira/browse/HBASE-3674?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
stack updated HBASE-3674: ------------------------- Attachment: 3674-distributed.txt How about this Prakash? This adds back the changes and sets default to true for the fail flag. > 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-distributed.txt, 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<Path> splitLog(final FileStatus[] logfiles) throws IOException > { > .... > outputSink.startWriterThreads(entryBuffers); > > try { > int i = 0; > for (FileStatus log : logfiles) { > Path logPath = log.getPath(); > long logLength = log.getLen(); > splitSize += logLength; > LOG.debug("Splitting hlog " + (i++ + 1) + " of " + logfiles.length > + ": " + logPath + ", length=" + 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 archive it > // before shutting down. This means the edits were persisted already > 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 + ". continuing"); > 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, conf); > > } finally { > splits = 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/.logs/sv2borg182,60020,1300384550664/sv2borg182%3A60020.1300461329481, > length=150 65662813573 2011-03-18 20:31:44,687 INFO > org.apache.hadoop.hbase.util.FSUtils: Recovering file > hdfs://sv2borg170:9000/hbase/.logs/sv2borg182,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]=000000cd000000502037383661376439656265643938636463343433386132343631323633303239371d6170695f6163636573735f746f6b656e5f7374 > > 6174735f6275636b65740000000d9fa4d5dc0000012ec9c7cbaf00ffffffff000000010000006d0000005d00000008002337626262663764626431616561366234616130656334383436653732333132643a32390764656661756c746170695f616e64726f69645f6c6f67676564 > > 696e5f73686172655f70656e64696e675f696e69740000012ec956b02804000000000000000100000000ffffffff4e128eca0eb078d0652b0abac467fd09000000cd000000502034663166613763666165333930666332653138346233393931303132623366331d6170695f6163 > > 636573735f746f6b656e5f73746174735f6275636b65740000000d9fa4d5dd0000012ec9c7cbaf00ffffffff000000010000006d0000005d00000008002366303734323966643036323862636530336238333938356239316237386633353a32390764656661756c746170695f61 > > 6e64726f69645f6c6f67676564696e5f73686172655f70656e64696e675f696e69740000012ec9569f1804000000000000000100000000000000d30000004e2066663763393964303633343339666531666461633761616632613964643631331b6170695f6163636573735f746f > 6b656e5f73746174735f68 > 813618 org.apache.hadoop.fs.ChecksumException: Checksum error: > /blk_7781725413191608261:of:/hbase/.logs/sv2borg182,60020,1300384550664/sv2borg182%3A60020.1300461329481 > at 15064576 > 813619 at > org.apache.hadoop.fs.FSInputChecker.verifySum(FSInputChecker.java:277) > 813620 at > org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:241) > 813621 at > org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:176) > 813622 at > org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:193) > 813623 at > org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158) > 813624 at > org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1175) > 813625 at > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1807) > 813626 at > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.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(DataOutputBuffer.java:63) > 813630 at > org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:101) > 813631 at > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1937) > 813632 at > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1837) > 813633 at > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1883) > 813634 at > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.next(SequenceFileLogReader.java:198) > 813635 at > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.next(SequenceFileLogReader.java:172) > 813636 at > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.parseHLog(HLogSplitter.java:429) > 813637 at > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLog(HLogSplitter.java:262) > 813638 at > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLog(HLogSplitter.java:188) > 813639 at > org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:197) > 813640 at > org.apache.hadoop.hbase.master.MasterFileSystem.splitLogAfterStartup(MasterFileSystem.java:181) > 813641 at > org.apache.hadoop.hbase.master.HMaster.finishInitialization(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: Found > Checksum error for blk_7781725413191608261_14589573 from 10.20.20.182:50010 > at 15064576 > 813644 2011-03-18 20:31:46,240 INFO org.apache.hadoop.hdfs.DFSClient: Could > not obtain block blk_7781725413191608261_14589573 from any node: > java.io.IOException: No live nodes contain current block. Will get new block > locations from namenode and retry... > 813645 2011-03-18 20:31:49,243 DEBUG > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Pushed=80624 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 block, > close out all writers tying off recovered.edits so far written. We'll skip > archiving these files because we only archive if all files are processed; 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 > -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira