diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index c551a94..708472d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -279,8 +279,6 @@ public class FSHLog implements WAL { private final int slowSyncNs; - private final static Object [] NO_ARGS = new Object []{}; - // If live datanode count is lower than the default replicas value, // RollWriter will be triggered in each sync(So the RollWriter will be // triggered one by one in a short time). Using it as a workaround to slow @@ -1332,8 +1330,8 @@ public class FSHLog implements WAL { } } } catch (Exception e) { - LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e + - " still proceeding ahead..."); + LOG.warn("DFSOutputStream.getNumCurrentReplicas failed because of " + e + + ", continuing..."); } return logRollNeeded; } @@ -1737,14 +1735,19 @@ public class FSHLog implements WAL { try { FSWALEntry entry = truck.unloadFSWALEntryPayload(); // If already an exception, do not try to append. Throw. - if (this.exception != null) throw this.exception; + if (this.exception != null) { + // We got an exception from an earlier attempt at append. Do not let this append + // go through. Fail it but stamp the sequenceid into the append because we + // need to do this to close the latch held down deep in WALKey... otherwise it will + // just hang out. The #append below does this too. + entry.stampRegionSequenceId(); + // Return to keep processing events coming off the ringbuffer + return; + } append(entry); } catch (Exception e) { - // Failed append. Record the exception. Throw it from here on out til new WAL in place - this.exception = new DamagedWALException(e); - // If append fails, presume any pending syncs will fail too; let all waiting handlers - // know of the exception. - cleanupOutstandingSyncsOnException(sequence, this.exception); + // Failed append. Record the exception. Throw it from here on out till new WAL in place. + this.exception = e; // Return to keep processing events coming off the ringbuffer return; } finally { @@ -1780,7 +1783,8 @@ public class FSHLog implements WAL { // sync a failed append. Fall through to the attainSafePoint below. It is part of the // means by which we put in place a new WAL. A new WAL is how we clean up. // Don't throw because then we'll not get to attainSafePoint. - cleanupOutstandingSyncsOnException(sequence, this.exception); + cleanupOutstandingSyncsOnException(sequence, + new DamagedWALException("On sync", this.exception)); } else { this.syncRunners[index].offer(sequence, this.syncFutures, this.syncFuturesCount); } @@ -1883,9 +1887,11 @@ public class FSHLog implements WAL { // Update metrics. postAppend(entry, EnvironmentEdgeManager.currentTime() - start); } catch (Exception e) { - LOG.warn("Could not append. Requesting close of WAL", e); + String msg = "Failed appending sequenceId=" + regionSequenceId + + ", requesting roll of WAL"; + LOG.warn(msg, e); requestLogRoll(); - throw e; + throw new DamagedWALException(msg, e); } numEntries.incrementAndGet(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java index 9b3dede..74284e0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java @@ -315,8 +315,12 @@ public class WALKey implements SequenceId, Comparable { */ public long getSequenceId(final long maxWaitForSeqId) throws IOException { // TODO: This implementation waiting on a latch is problematic because if a higher level - // determines we should stop or abort, there is not global list of all these blocked WALKeys - // waiting on a sequence id; they can't be cancelled... interrupted. See getNextSequenceId + // determines we should stop or abort, there is no global list of all these blocked WALKeys + // waiting on a sequence id; they can't be cancelled... interrupted. See getNextSequenceId. + // + // UPDATE: I think we can remove the timeout now we are stamping all walkeys with sequenceid, + // even those that have failed (previously we were not... so they would just hang out...). + // St.Ack 20150910 try { if (maxWaitForSeqId < 0) { this.seqNumAssignedLatch.await(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java index a620951..ccf2b15 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; @@ -81,6 +82,8 @@ public class TestWALLockup { public void setup() throws IOException { TEST_UTIL = HBaseTestingUtility.createLocalHTU(); CONF = TEST_UTIL.getConfiguration(); + // Disable block cache. + CONF.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); dir = TEST_UTIL.getDataTestDir("TestHRegion").toString(); tableName = TableName.valueOf(name.getMethodName()); } @@ -139,10 +142,10 @@ public class TestWALLockup { protected void beforeWaitOnSafePoint() { if (throwException) { LOG.info("COUNTDOWN"); - // Don't countdown latch until someone waiting on it. - while (this.latch.getCount() <= 0) { - Threads.sleep(10); - } + // Don't countdown latch until someone waiting on it otherwise, the above + // afterCreatingZigZagLatch will get to the latch and no one will ever free it and we'll + // be stuck; test won't go down + while (this.latch.getCount() <= 0) Threads.sleep(1); this.latch.countDown(); } }