HBASE-5029 TestDistributedLogSplitting fails on occasion; Added catch of NPE and reenabled ignored test
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1220991 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2062ea1d00
commit
1222383f1d
|
@ -211,7 +211,12 @@ public class SequenceFileLogWriter implements HLog.Writer {
|
|||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.writer != null) {
|
||||
this.writer.close();
|
||||
try {
|
||||
this.writer.close();
|
||||
} catch (NullPointerException npe) {
|
||||
// Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
|
||||
LOG.warn(npe);
|
||||
}
|
||||
this.writer = null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -254,7 +254,6 @@ public class TestDistributedLogSplitting {
|
|||
* detects that the region server has aborted.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Ignore
|
||||
@Test (timeout=300000)
|
||||
public void testWorkerAbort() throws Exception {
|
||||
LOG.info("testWorkerAbort");
|
||||
|
|
Loading…
Reference in New Issue