[ https://issues.apache.org/jira/browse/HBASE-5235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13189969#comment-13189969 ]
ramkrishna.s.vasudevan commented on HBASE-5235: ----------------------------------------------- But Ted, in closeStreams() we also do the steps related to renaming the .temp files in recovered.edits. So we should only do the {code} for (WriterAndPath wap : logWriters.values()) { try { wap.w.close(); } catch (IOException ioe) { LOG.error("Couldn't close log at " + wap.p, ioe); thrown.add(ioe); continue; } {code} and make the master abort so that the subsequent split can parse the HLog. Correct me if am wrong. > HLogSplitter writer thread's streams not getting closed when any of the > writer threads has exceptions. > ------------------------------------------------------------------------------------------------------ > > Key: HBASE-5235 > URL: https://issues.apache.org/jira/browse/HBASE-5235 > Project: HBase > Issue Type: Bug > Affects Versions: 0.92.0, 0.90.5 > Reporter: ramkrishna.s.vasudevan > Assignee: ramkrishna.s.vasudevan > Fix For: 0.92.1, 0.90.6 > > > Pls find the analysis. Correct me if am wrong > {code} > 2012-01-15 05:14:02,374 FATAL > org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: WriterThread-9 Got > while writing log entry to log > java.io.IOException: All datanodes 10.18.40.200:50010 are bad. Aborting... > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:3373) > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(DFSClient.java:2811) > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:3026) > {code} > Here we have an exception in one of the writer threads. If any exception we > try to hold it in an Atomic variable > {code} > private void writerThreadError(Throwable t) { > thrown.compareAndSet(null, t); > } > {code} > In the finally block of splitLog we try to close the streams. > {code} > for (WriterThread t: writerThreads) { > try { > t.join(); > } catch (InterruptedException ie) { > throw new IOException(ie); > } > checkForErrors(); > } > LOG.info("Split writers finished"); > > return closeStreams(); > {code} > Inside checkForErrors > {code} > private void checkForErrors() throws IOException { > Throwable thrown = this.thrown.get(); > if (thrown == null) return; > if (thrown instanceof IOException) { > throw (IOException)thrown; > } else { > throw new RuntimeException(thrown); > } > } > So once we throw the exception the DFSStreamer threads are not getting closed. > {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira