You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2016/03/26 08:42:26 UTC

[1/4] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Repository: hbase
Updated Branches:
  refs/heads/master 1a9837ab7 -> c96b642f1


http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
index 39743d5..41a790e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java
@@ -27,12 +27,15 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
@@ -51,7 +54,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
  * Hadoop serialization).
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class SequenceFileLogWriter extends WriterBase {
+public class SequenceFileLogWriter implements DefaultWALProvider.Writer {
   private static final Log LOG = LogFactory.getLog(SequenceFileLogWriter.class);
   // The sequence file we delegate to.
   private SequenceFile.Writer writer;
@@ -59,6 +62,8 @@ public class SequenceFileLogWriter extends WriterBase {
   // in the SequenceFile.Writer 'writer' instance above.
   private FSDataOutputStream writer_out;
 
+  private CompressionContext compressionContext;
+
   // Legacy stuff from pre-PB WAL metadata.
   private static final Text WAL_VERSION_KEY = new Text("version");
   private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
@@ -88,10 +93,23 @@ public class SequenceFileLogWriter extends WriterBase {
     return new Metadata(metaMap);
   }
 
+  private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
+    boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
+    if (doCompress) {
+      try {
+        this.compressionContext = new CompressionContext(LRUDictionary.class,
+            FSUtils.isRecoveredEdits(path), conf.getBoolean(
+                CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
+      } catch (Exception e) {
+        throw new IOException("Failed to initiate CompressionContext", e);
+      }
+    }
+    return doCompress;
+  }
+
   @Override
   public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
   throws IOException {
-    super.init(fs, path, conf, overwritable);
     boolean compress = initializeCompressionContext(conf, path);
     // Create a SF.Writer instance.
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java
new file mode 100644
index 0000000..bedb915
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRollPeriod.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncLogRollPeriod extends TestLogRollPeriod {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestLogRollPeriod.TEST_UTIL.getConfiguration();
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    TestLogRollPeriod.setUpBeforeClass();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
new file mode 100644
index 0000000..fabf6d2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ VerySlowRegionServerTests.class, LargeTests.class })
+public class TestAsyncLogRolling extends AbstractTestLogRolling {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestAsyncLogRolling.TEST_UTIL.getConfiguration();
+    conf.setInt(AsyncFSWAL.ASYNC_WAL_CREATE_MAX_RETRIES, 100);
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    AbstractTestLogRolling.setUpBeforeClass();
+  }
+
+  @Test(timeout = 180000)
+  public void testLogRollOnDatanodeDeath() throws IOException, InterruptedException {
+    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 3, true, null, null);
+    tableName = getName();
+    Table table = createTestTable(tableName);
+    TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
+    doPut(table, 1);
+    server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
+    HRegionInfo hri = server.getOnlineRegions(table.getName()).get(0).getRegionInfo();
+    AsyncFSWAL wal = (AsyncFSWAL) server.getWAL(hri);
+    int numRolledLogFiles = AsyncFSWALProvider.getNumRolledLogFiles(wal);
+    DatanodeInfo[] dnInfos = wal.getPipeline();
+    DataNodeProperties dnProp = TEST_UTIL.getDFSCluster().stopDataNode(dnInfos[0].getName());
+    TEST_UTIL.getDFSCluster().restartDataNode(dnProp);
+    doPut(table, 2);
+    assertEquals(numRolledLogFiles + 1, AsyncFSWALProvider.getNumRolledLogFiles(wal));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
new file mode 100644
index 0000000..7d6c6d9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputFlushHandler;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.base.Throwables;
+
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncProtobufLog extends AbstractTestProtobufLog<WALProvider.AsyncWriter> {
+
+  private static EventLoopGroup EVENT_LOOP_GROUP;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    EVENT_LOOP_GROUP = new NioEventLoopGroup();
+    AbstractTestProtobufLog.setUpBeforeClass();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    AbstractTestProtobufLog.tearDownAfterClass();
+    EVENT_LOOP_GROUP.shutdownGracefully().syncUninterruptibly();
+  }
+
+  @Override
+  protected AsyncWriter createWriter(Path path) throws IOException {
+    return AsyncFSWALProvider.createAsyncWriter(TEST_UTIL.getConfiguration(), fs, path, false,
+      EVENT_LOOP_GROUP.next());
+  }
+
+  @Override
+  protected void append(AsyncWriter writer, Entry entry) throws IOException {
+    writer.append(entry);
+  }
+
+  @Override
+  protected void sync(AsyncWriter writer) throws IOException {
+    FanOutOneBlockAsyncDFSOutputFlushHandler handler = new FanOutOneBlockAsyncDFSOutputFlushHandler();
+    writer.sync(handler, null);
+    try {
+      handler.get();
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    } catch (ExecutionException e) {
+      Throwables.propagateIfPossible(e.getCause(), IOException.class);
+      throw new IOException(e.getCause());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
new file mode 100644
index 0000000..ca415fd
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncWALReplay extends TestWALReplay {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    TestWALReplay.setUpBeforeClass();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java
new file mode 100644
index 0000000..3b8869b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplayCompressed.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestAsyncWALReplayCompressed extends TestWALReplay {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
+    conf.set(WALFactory.WAL_PROVIDER, "asyncfs");
+    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+    TestWALReplay.setUpBeforeClass();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index 0662716..5783106 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -30,28 +31,35 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  * Tests for WAL write durability
  */
+@RunWith(Parameterized.class)
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestDurability {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -64,6 +72,13 @@ public class TestDurability {
   private static byte[] ROW = Bytes.toBytes("row");
   private static byte[] COL = Bytes.toBytes("col");
 
+  @Parameter
+  public String walProvider;
+
+  @Parameters(name = "{index}: provider={0}")
+  public static Iterable<Object[]> data() {
+    return Arrays.asList(new Object[] { "defaultProvider" }, new Object[] { "asyncfs" });
+  }
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -81,6 +96,16 @@ public class TestDurability {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  @Before
+  public void setUp() {
+    CONF.set(WALFactory.WAL_PROVIDER, walProvider);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    FS.delete(DIR, true);
+  }
+
   @Test
   public void testDurability() throws Exception {
     final WALFactory wals = new WALFactory(CONF, null, "TestDurability");

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index c05e7f0..b7c1c73 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -65,11 +65,11 @@ import org.junit.experimental.categories.Category;
  */
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestLogRollAbort {
-  private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
+  private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
   private static MiniDFSCluster dfsCluster;
   private static Admin admin;
   private static MiniHBaseCluster cluster;
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   /* For the split-then-roll test */
   private static final Path HBASEDIR = new Path("/hbase");
@@ -212,7 +212,7 @@ public class TestLogRollAbort {
       }
       // Send the data to HDFS datanodes and close the HDFS writer
       log.sync();
-      ((FSHLog) log).replaceWriter(((FSHLog)log).getOldPath(), null, null, null);
+      ((AbstractFSWAL<?>) log).replaceWriter(((FSHLog)log).getOldPath(), null, null);
 
       /* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog()
        * handles RS shutdowns (as observed by the splitting process)

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
index 1bf686f..1141871 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java
@@ -19,21 +19,21 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertFalse;
 
-import java.util.List;
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,9 +44,9 @@ import org.junit.experimental.categories.Category;
  */
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestLogRollPeriod {
-  private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
+  private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private final static long LOG_ROLL_PERIOD = 4000;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
index 3ab49c0..37b23e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,7 +23,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.EOFException;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -33,275 +31,70 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
 
-/**
- * Test log deletion as logs are rolled.
- */
-@Category({VerySlowRegionServerTests.class, LargeTests.class})
-public class TestLogRolling  {
+@Category({ VerySlowRegionServerTests.class, LargeTests.class })
+public class TestLogRolling extends AbstractTestLogRolling {
+
   private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
-  private HRegionServer server;
-  private String tableName;
-  private byte[] value;
-  private FileSystem fs;
-  private MiniDFSCluster dfsCluster;
-  private Admin admin;
-  private MiniHBaseCluster cluster;
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  @Rule public final TestName name = new TestName();
-
-  public TestLogRolling()  {
-    this.server = null;
-    this.tableName = null;
-
-    String className = this.getClass().getName();
-    StringBuilder v = new StringBuilder(className);
-    while (v.length() < 1000) {
-      v.append(className);
-    }
-    this.value = Bytes.toBytes(v.toString());
-  }
 
-  // Need to override this setup so we can edit the config before it gets sent
-  // to the HDFS & HBase cluster startup.
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     // TODO: testLogRollOnDatanodeDeath fails if short circuit reads are on under the hadoop2
     // profile. See HBASE-9337 for related issues.
     System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
 
-    /**** configuration for testLogRolling ****/
-    // Force a region split after every 768KB
-    TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 768L * 1024L);
-
-    // We roll the log after every 32 writes
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32);
-
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.errors.tolerated", 2);
-    TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000);
-
-    // For less frequently updated regions flush after every 2 flushes
-    TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
-
-    // We flush the cache after every 8192 bytes
-    TEST_UTIL.getConfiguration().setInt(
-        HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
-
-    // Increase the amount of time between client retries
-    TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
-
-    // Reduce thread wake frequency so that other threads can get
-    // a chance to run.
-    TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
-
-   /**** configuration for testLogRollOnDatanodeDeath ****/
-   // make sure log.hflush() calls syncFs() to open a pipeline
+    /**** configuration for testLogRollOnDatanodeDeath ****/
+    // make sure log.hflush() calls syncFs() to open a pipeline
     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
-   // lower the namenode & datanode heartbeat so the namenode
-   // quickly detects datanode failures
+    // lower the namenode & datanode heartbeat so the namenode
+    // quickly detects datanode failures
     TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
     TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
     // the namenode might still try to choose the recently-dead datanode
     // for a pipeline, so try to a new pipeline multiple times
-     TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30);
+    TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30);
     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.tolerable.lowreplication", 2);
     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.hlog.lowreplication.rolllimit", 3);
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    TEST_UTIL.startMiniCluster(1, 1, 2);
-
-    cluster = TEST_UTIL.getHBaseCluster();
-    dfsCluster = TEST_UTIL.getDFSCluster();
-    fs = TEST_UTIL.getTestFileSystem();
-    admin = TEST_UTIL.getHBaseAdmin();
-
-    // disable region rebalancing (interferes with log watching)
-    cluster.getMaster().balanceSwitch(false);
-  }
-
-  @After
-  public void tearDown() throws Exception  {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  private void startAndWriteData() throws IOException, InterruptedException {
-    // When the hbase:meta table can be opened, the region servers are running
-    TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-    this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
-
-    Table table = createTestTable(this.tableName);
-
-    server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
-    for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
-      doPut(table, i);
-      if (i % 32 == 0) {
-        // After every 32 writes sleep to let the log roller run
-        try {
-          Thread.sleep(2000);
-        } catch (InterruptedException e) {
-          // continue
-        }
-      }
-    }
-  }
-
-  /**
-   * Tests that log rolling doesn't hang when no data is written.
-   */
-  @Test(timeout=120000)
-  public void testLogRollOnNothingWritten() throws Exception {
-    final Configuration conf = TEST_UTIL.getConfiguration();
-    final WALFactory wals = new WALFactory(conf, null,
-        ServerName.valueOf("test.com",8080, 1).toString());
-    final WAL newLog = wals.getWAL(new byte[]{}, null);
-    try {
-      // Now roll the log before we write anything.
-      newLog.rollWriter(true);
-    } finally {
-      wals.close();
-    }
-  }
-
-  /**
-   * Tests that logs are deleted
-   * @throws IOException
-   * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
-   */
-  @Test
-  public void testLogRolling() throws Exception {
-    this.tableName = getName();
-    // TODO: Why does this write data take for ever?
-    startAndWriteData();
-    HRegionInfo region =
-        server.getOnlineRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
-    final WAL log = server.getWAL(region);
-    LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
-        " log files");
-
-      // flush all regions
-      for (Region r: server.getOnlineRegionsLocalContext()) {
-        r.flush(true);
-      }
-
-      // Now roll the log
-      log.rollWriter();
-
-    int count = DefaultWALProvider.getNumRolledLogFiles(log);
-    LOG.info("after flushing all regions and rolling logs there are " + count + " log files");
-      assertTrue(("actual count: " + count), count <= 2);
-  }
-
-  private String getName() {
-    return "TestLogRolling-" + name.getMethodName();
-  }
-
-  void writeData(Table table, int rownum) throws IOException {
-    doPut(table, rownum);
-
-    // sleep to let the log roller run (if it needs to)
-    try {
-      Thread.sleep(2000);
-    } catch (InterruptedException e) {
-      // continue
-    }
-  }
-
-  void validateData(Table table, int rownum) throws IOException {
-    String row = "row" + String.format("%1$04d", rownum);
-    Get get = new Get(Bytes.toBytes(row));
-    get.addFamily(HConstants.CATALOG_FAMILY);
-    Result result = table.get(get);
-    assertTrue(result.size() == 1);
-    assertTrue(Bytes.equals(value,
-                result.getValue(HConstants.CATALOG_FAMILY, null)));
-    LOG.info("Validated row " + row);
-  }
-
-  void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout)
-      throws IOException {
-    for (int i = 0; i < 10; i++) {
-      Put put = new Put(Bytes.toBytes("row"
-          + String.format("%1$04d", (start + i))));
-      put.addColumn(HConstants.CATALOG_FAMILY, null, value);
-      table.put(put);
-    }
-    Put tmpPut = new Put(Bytes.toBytes("tmprow"));
-    tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value);
-    long startTime = System.currentTimeMillis();
-    long remaining = timeout;
-    while (remaining > 0) {
-      if (log.isLowReplicationRollEnabled() == expect) {
-        break;
-      } else {
-        // Trigger calling FSHlog#checkLowReplication()
-        table.put(tmpPut);
-        try {
-          Thread.sleep(200);
-        } catch (InterruptedException e) {
-          // continue
-        }
-        remaining = timeout - (System.currentTimeMillis() - startTime);
-      }
-    }
+    AbstractTestLogRolling.setUpBeforeClass();
   }
 
   /**
-   * Tests that logs are rolled upon detecting datanode death
-   * Requires an HDFS jar with HDFS-826 & syncFs() support (HDFS-200)
+   * Tests that logs are rolled upon detecting datanode death Requires an HDFS jar with HDFS-826 &
+   * syncFs() support (HDFS-200)
    */
   @Test
   public void testLogRollOnDatanodeDeath() throws Exception {
     TEST_UTIL.ensureSomeRegionServersAvailable(2);
     assertTrue("This test requires WAL file replication set to 2.",
       fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) == 2);
-    LOG.info("Replication=" +
-      fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
+    LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
 
     this.server = cluster.getRegionServer(0);
 
@@ -328,29 +121,28 @@ public class TestLogRolling  {
     });
 
     // don't run this test without append support (HDFS-200 & HDFS-142)
-    assertTrue("Need append support for this test", FSUtils
-        .isAppendSupported(TEST_UTIL.getConfiguration()));
+    assertTrue("Need append support for this test",
+      FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
 
     // add up the datanode count, to ensure proper replication when we kill 1
     // This function is synchronous; when it returns, the dfs cluster is active
     // We start 3 servers and then stop 2 to avoid a directory naming conflict
-    //  when we stop/start a namenode later, as mentioned in HBASE-5163
+    // when we stop/start a namenode later, as mentioned in HBASE-5163
     List<DataNode> existingNodes = dfsCluster.getDataNodes();
     int numDataNodes = 3;
-    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true,
-        null, null);
+    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), numDataNodes, true, null, null);
     List<DataNode> allNodes = dfsCluster.getDataNodes();
-    for (int i = allNodes.size()-1; i >= 0; i--) {
+    for (int i = allNodes.size() - 1; i >= 0; i--) {
       if (existingNodes.contains(allNodes.get(i))) {
-        dfsCluster.stopDataNode( i );
+        dfsCluster.stopDataNode(i);
       }
     }
 
-    assertTrue("DataNodes " + dfsCluster.getDataNodes().size() +
-        " default replication " +
-        fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
-    dfsCluster.getDataNodes().size() >=
-      fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
+    assertTrue(
+      "DataNodes " + dfsCluster.getDataNodes().size() + " default replication "
+          + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()),
+      dfsCluster.getDataNodes()
+          .size() >= fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) + 1);
 
     writeData(table, 2);
 
@@ -358,13 +150,12 @@ public class TestLogRolling  {
     LOG.info("log.getCurrentFileName(): " + log.getCurrentFileName());
     long oldFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
     assertTrue("Log should have a timestamp older than now",
-        curTime > oldFilenum && oldFilenum != -1);
+      curTime > oldFilenum && oldFilenum != -1);
 
     assertTrue("The log shouldn't have rolled yet",
-        oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
-    final DatanodeInfo[] pipeline = log.getPipeLine();
-    assertTrue(pipeline.length ==
-        fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
+      oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
+    final DatanodeInfo[] pipeline = log.getPipeline();
+    assertTrue(pipeline.length == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
 
     // kill a datanode in the pipeline to force a log roll on the next sync()
     // This function is synchronous, when it returns the node is killed.
@@ -375,41 +166,38 @@ public class TestLogRolling  {
     long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
 
     assertTrue("Missing datanode should've triggered a log roll",
-        newFilenum > oldFilenum && newFilenum > curTime);
+      newFilenum > oldFilenum && newFilenum > curTime);
 
     assertTrue("The log rolling hook should have been called with the low replication flag",
-        lowReplicationHookCalled.get());
+      lowReplicationHookCalled.get());
 
     // write some more log data (this should use a new hdfs_out)
     writeData(table, 3);
     assertTrue("The log should not roll again.",
-        DefaultWALProvider.extractFileNumFromWAL(log) == newFilenum);
+      DefaultWALProvider.extractFileNumFromWAL(log) == newFilenum);
     // kill another datanode in the pipeline, so the replicas will be lower than
     // the configured value 2.
     assertTrue(dfsCluster.stopDataNode(pipeline[1].getName()) != null);
 
     batchWriteAndWait(table, log, 3, false, 14000);
     int replication = log.getLogReplication();
-    assertTrue("LowReplication Roller should've been disabled, current replication="
-            + replication, !log.isLowReplicationRollEnabled());
+    assertTrue("LowReplication Roller should've been disabled, current replication=" + replication,
+      !log.isLowReplicationRollEnabled());
 
-    dfsCluster
-        .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
+    dfsCluster.startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
 
     // Force roll writer. The new log file will have the default replications,
     // and the LowReplication Roller will be enabled.
     log.rollWriter(true);
     batchWriteAndWait(table, log, 13, true, 10000);
     replication = log.getLogReplication();
-    assertTrue("New log file should have the default replication instead of " +
-      replication,
+    assertTrue("New log file should have the default replication instead of " + replication,
       replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
     assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled());
   }
 
   /**
-   * Test that WAL is rolled when all data nodes in the pipeline have been
-   * restarted.
+   * Test that WAL is rolled when all data nodes in the pipeline have been restarted.
    * @throws Exception
    */
   @Test
@@ -417,8 +205,7 @@ public class TestLogRolling  {
     LOG.info("Starting testLogRollOnPipelineRestart");
     assertTrue("This test requires WAL file replication.",
       fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1);
-    LOG.info("Replication=" +
-      fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
+    LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
     // When the hbase:meta table can be opened, the region servers are running
     Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
     try {
@@ -441,10 +228,11 @@ public class TestLogRolling  {
       log.registerWALActionsListener(new WALActionsListener.Base() {
 
         @Override
-        public void preLogRoll(Path oldFile, Path newFile)  {
-          LOG.debug("preLogRoll: oldFile="+oldFile+" newFile="+newFile);
+        public void preLogRoll(Path oldFile, Path newFile) {
+          LOG.debug("preLogRoll: oldFile=" + oldFile + " newFile=" + newFile);
           preLogRolledCalled.add(new Integer(1));
         }
+
         @Override
         public void postLogRoll(Path oldFile, Path newFile) {
           paths.add(newFile);
@@ -452,8 +240,8 @@ public class TestLogRolling  {
       });
 
       // don't run this test without append support (HDFS-200 & HDFS-142)
-      assertTrue("Need append support for this test", FSUtils
-          .isAppendSupported(TEST_UTIL.getConfiguration()));
+      assertTrue("Need append support for this test",
+        FSUtils.isAppendSupported(TEST_UTIL.getConfiguration()));
 
       writeData(table, 1002);
 
@@ -461,10 +249,10 @@ public class TestLogRolling  {
       LOG.info("log.getCurrentFileName()): " + DefaultWALProvider.getCurrentFileName(log));
       long oldFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
       assertTrue("Log should have a timestamp older than now",
-          curTime > oldFilenum && oldFilenum != -1);
+        curTime > oldFilenum && oldFilenum != -1);
 
-      assertTrue("The log shouldn't have rolled yet", oldFilenum ==
-          DefaultWALProvider.extractFileNumFromWAL(log));
+      assertTrue("The log shouldn't have rolled yet",
+        oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
 
       // roll all datanodes in the pipeline
       dfsCluster.restartDataNodes();
@@ -478,7 +266,7 @@ public class TestLogRolling  {
       long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
 
       assertTrue("Missing datanode should've triggered a log roll",
-          newFilenum > oldFilenum && newFilenum > curTime);
+        newFilenum > oldFilenum && newFilenum > curTime);
       validateData(table, 1003);
 
       writeData(table, 1004);
@@ -496,30 +284,30 @@ public class TestLogRolling  {
       // force a log roll to read back and verify previously written logs
       log.rollWriter(true);
       assertTrue("preLogRolledCalled has size of " + preLogRolledCalled.size(),
-          preLogRolledCalled.size() >= 1);
+        preLogRolledCalled.size() >= 1);
 
       // read back the data written
       Set<String> loggedRows = new HashSet<String>();
       FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration());
       for (Path p : paths) {
         LOG.debug("recovering lease for " + p);
-        fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p,
-          TEST_UTIL.getConfiguration(), null);
+        fsUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(),
+          null);
 
-        LOG.debug("Reading WAL "+FSUtils.getPath(p));
+        LOG.debug("Reading WAL " + FSUtils.getPath(p));
         WAL.Reader reader = null;
         try {
           reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration());
           WAL.Entry entry;
           while ((entry = reader.next()) != null) {
-            LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getCells());
+            LOG.debug("#" + entry.getKey().getSequenceId() + ": " + entry.getEdit().getCells());
             for (Cell cell : entry.getEdit().getCells()) {
-              loggedRows.add(Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(),
-                cell.getRowLength()));
+              loggedRows.add(
+                Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
             }
           }
         } catch (EOFException e) {
-          LOG.debug("EOF reading file "+FSUtils.getPath(p));
+          LOG.debug("EOF reading file " + FSUtils.getPath(p));
         } finally {
           if (reader != null) reader.close();
         }
@@ -532,7 +320,7 @@ public class TestLogRolling  {
       assertTrue(loggedRows.contains("row1005"));
 
       // flush all regions
-      for (Region r: server.getOnlineRegionsLocalContext()) {
+      for (Region r : server.getOnlineRegionsLocalContext()) {
         try {
           r.flush(true);
         } catch (Exception e) {
@@ -547,19 +335,19 @@ public class TestLogRolling  {
 
       ResultScanner scanner = table.getScanner(new Scan());
       try {
-        for (int i=2; i<=5; i++) {
+        for (int i = 2; i <= 5; i++) {
           Result r = scanner.next();
           assertNotNull(r);
           assertFalse(r.isEmpty());
-          assertEquals("row100"+i, Bytes.toString(r.getRow()));
+          assertEquals("row100" + i, Bytes.toString(r.getRow()));
         }
       } finally {
         scanner.close();
       }
 
       // verify that no region servers aborted
-      for (JVMClusterUtil.RegionServerThread rsThread:
-        TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
+      for (JVMClusterUtil.RegionServerThread rsThread : TEST_UTIL.getHBaseCluster()
+          .getRegionServerThreads()) {
         assertFalse(rsThread.getRegionServer().isAborted());
       }
     } finally {
@@ -567,80 +355,4 @@ public class TestLogRolling  {
     }
   }
 
-  /**
-   * Tests that logs are deleted when some region has a compaction
-   * record in WAL and no other records. See HBASE-8597.
-   */
-  @Test
-  public void testCompactionRecordDoesntBlockRolling() throws Exception {
-    Table table = null;
-
-    // When the hbase:meta table can be opened, the region servers are running
-    Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-    try {
-      table = createTestTable(getName());
-
-      server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
-      Region region = server.getOnlineRegions(table.getName()).get(0);
-      final WAL log = server.getWAL(region.getRegionInfo());
-      Store s = region.getStore(HConstants.CATALOG_FAMILY);
-
-      //have to flush namespace to ensure it doesn't affect wall tests
-      admin.flush(TableName.NAMESPACE_TABLE_NAME);
-
-      // Put some stuff into table, to make sure we have some files to compact.
-      for (int i = 1; i <= 2; ++i) {
-        doPut(table, i);
-        admin.flush(table.getName());
-      }
-      doPut(table, 3); // don't flush yet, or compaction might trigger before we roll WAL
-      assertEquals("Should have no WAL after initial writes", 0,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-      assertEquals(2, s.getStorefilesCount());
-
-      // Roll the log and compact table, to have compaction record in the 2nd WAL.
-      log.rollWriter();
-      assertEquals("Should have WAL; one table is not flushed", 1,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-      admin.flush(table.getName());
-      region.compact(false);
-      // Wait for compaction in case if flush triggered it before us.
-      Assert.assertNotNull(s);
-      for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) {
-        Threads.sleepWithoutInterrupt(200);
-      }
-      assertEquals("Compaction didn't happen", 1, s.getStorefilesCount());
-
-      // Write some value to the table so the WAL cannot be deleted until table is flushed.
-      doPut(table, 0); // Now 2nd WAL will have both compaction and put record for table.
-      log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
-      assertEquals("Should have WAL; one table is not flushed", 1,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-
-      // Flush table to make latest WAL obsolete; write another record, and roll again.
-      admin.flush(table.getName());
-      doPut(table, 1);
-      log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
-      assertEquals("Should have 1 WALs at the end", 1,
-          DefaultWALProvider.getNumRolledLogFiles(log));
-    } finally {
-      if (t != null) t.close();
-      if (table != null) table.close();
-    }
-  }
-
-  private void doPut(Table table, int i) throws IOException {
-    Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
-    put.addColumn(HConstants.CATALOG_FAMILY, null, value);
-    table.put(put);
-  }
-
-  private Table createTestTable(String tableName) throws IOException {
-    // Create the test table and open it
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
-    desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc);
-    return TEST_UTIL.getConnection().getTable(desc.getTableName());
-  }
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
index e6237f8..61ee589 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,190 +17,32 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WALProvider;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-/**
- * WAL tests that can be reused across providers.
- */
-@Category({RegionServerTests.class, MediumTests.class})
-public class TestProtobufLog {
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  protected FileSystem fs;
-  protected Path dir;
-  protected WALFactory wals;
-
-  @Rule
-  public final TestName currentTest = new TestName();
 
-  @Before
-  public void setUp() throws Exception {
-    fs = TEST_UTIL.getDFSCluster().getFileSystem();
-    dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
-    wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    wals.close();
-    FileStatus[] entries = fs.listStatus(new Path("/"));
-    for (FileStatus dir : entries) {
-      fs.delete(dir.getPath(), true);
-    }
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Make block sizes small.
-    TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
-    // needed for testAppendClose()
-    TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
-    TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
-    // quicker heartbeat interval for faster DN death notification
-    TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
-    TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
-    TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer> {
 
-    // faster failover with cluster.shutdown();fs.close() idiom
-    TEST_UTIL.getConfiguration()
-        .setInt("hbase.ipc.client.connect.max.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-        "dfs.client.block.recovery.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-      "hbase.ipc.client.connection.maxidletime", 500);
-    TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
-        SampleRegionWALObserver.class.getName());
-    TEST_UTIL.startMiniDFSCluster(3);
+  @Override
+  protected Writer createWriter(Path path) throws IOException {
+    return DefaultWALProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false);
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+  @Override
+  protected void append(Writer writer, Entry entry) throws IOException {
+    writer.append(entry);
   }
 
-  /**
-   * Reads the WAL with and without WALTrailer.
-   * @throws IOException
-   */
-  @Test
-  public void testWALTrailer() throws IOException {
-    // read With trailer.
-    doRead(true);
-    // read without trailer
-    doRead(false);
-  }
-
-  /**
-   * Appends entries in the WAL and reads it.
-   * @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
-   *          so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
-   *          call. This means that reader is not aware of the trailer. In this scenario, if the
-   *          reader tries to read the trailer in its next() call, it returns false from
-   *          ProtoBufLogReader.
-   * @throws IOException
-   */
-  private void doRead(boolean withTrailer) throws IOException {
-    final int columnCount = 5;
-    final int recordCount = 5;
-    final TableName tableName =
-        TableName.valueOf("tablename");
-    final byte[] row = Bytes.toBytes("row");
-    long timestamp = System.currentTimeMillis();
-    Path path = new Path(dir, "tempwal");
-    // delete the log if already exists, for test only
-    fs.delete(path, true);
-    WALProvider.Writer writer = null;
-    ProtobufLogReader reader = null;
-    try {
-      HRegionInfo hri = new HRegionInfo(tableName,
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      fs.mkdirs(dir);
-      // Write log in pb format.
-      writer = wals.createWALWriter(fs, path);
-      for (int i = 0; i < recordCount; ++i) {
-        WALKey key = new WALKey(
-            hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
-        WALEdit edit = new WALEdit();
-        for (int j = 0; j < columnCount; ++j) {
-          if (i == 0) {
-            htd.addFamily(new HColumnDescriptor("column" + j));
-          }
-          String value = i + "" + j;
-          edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
-        }
-        writer.append(new WAL.Entry(key, edit));
-      }
-      writer.sync();
-      if (withTrailer) writer.close();
-
-      // Now read the log using standard means.
-      reader = (ProtobufLogReader) wals.createReader(fs, path);
-      if (withTrailer) {
-        assertNotNull(reader.trailer);
-      } else {
-        assertNull(reader.trailer);
-      }
-      for (int i = 0; i < recordCount; ++i) {
-        WAL.Entry entry = reader.next();
-        assertNotNull(entry);
-        assertEquals(columnCount, entry.getEdit().size());
-        assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
-        assertEquals(tableName, entry.getKey().getTablename());
-        int idx = 0;
-        for (Cell val : entry.getEdit().getCells()) {
-          assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
-            val.getRowLength()));
-          String value = i + "" + idx;
-          assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
-          idx++;
-        }
-      }
-      WAL.Entry entry = reader.next();
-      assertNull(entry);
-    } finally {
-      if (writer != null) {
-        writer.close();
-      }
-      if (reader != null) {
-        reader.close();
-      }
-    }
+  @Override
+  protected void sync(Writer writer) throws IOException {
+    writer.sync();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
index 4987fd4..b225554 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplayCompressed.java
@@ -36,5 +36,4 @@ public class TestWALReplayCompressed extends TestWALReplay {
     Configuration conf = TestWALReplay.TEST_UTIL.getConfiguration();
     conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
   }
-
 }


[2/4] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 06318f0..24f195d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -48,17 +48,20 @@ import org.apache.hadoop.hbase.wal.WALKey;
 class FSWALEntry extends Entry {
   // The below data members are denoted 'transient' just to highlight these are not persisted;
   // they are only in memory and held here while passing over the ring buffer.
-  private final transient long sequence;
+  private final transient long txid;
   private final transient boolean inMemstore;
   private final transient HRegionInfo hri;
-  private final Set<byte[]> familyNames;
+  private final transient Set<byte[]> familyNames;
+  // In the new WAL logic, we will rewrite failed WAL entries to new WAL file, so we need to avoid
+  // calling stampRegionSequenceId again.
+  private transient boolean stamped = false;
 
-  FSWALEntry(final long sequence, final WALKey key, final WALEdit edit,
+  FSWALEntry(final long txid, final WALKey key, final WALEdit edit,
       final HRegionInfo hri, final boolean inMemstore) {
     super(key, edit);
     this.inMemstore = inMemstore;
     this.hri = hri;
-    this.sequence = sequence;
+    this.txid = txid;
     if (inMemstore) {
       // construct familyNames here to reduce the work of log sinker.
       ArrayList<Cell> cells = this.getEdit().getCells();
@@ -80,7 +83,7 @@ class FSWALEntry extends Entry {
   }
 
   public String toString() {
-    return "sequence=" + this.sequence + ", " + super.toString();
+    return "sequence=" + this.txid + ", " + super.toString();
   };
 
   boolean isInMemstore() {
@@ -92,10 +95,10 @@ class FSWALEntry extends Entry {
   }
 
   /**
-   * @return The sequence on the ring buffer when this edit was added.
+   * @return The transaction id of this edit.
    */
-  long getSequence() {
-    return this.sequence;
+  long getTxid() {
+    return this.txid;
   }
 
   /**
@@ -103,9 +106,12 @@ class FSWALEntry extends Entry {
    * SIDE-EFFECT is our stamping the sequenceid into every Cell AND setting the sequenceid into the
    * MVCC WriteEntry!!!!
    * @return The sequenceid we stamped on this edit.
-   * @throws IOException
    */
   long stampRegionSequenceId() throws IOException {
+    if (stamped) {
+      return getKey().getSequenceId();
+    }
+    stamped = true;
     long regionSequenceId = WALKey.NO_SEQUENCE_ID;
     MultiVersionConcurrencyControl mvcc = getKey().getMvcc();
     MultiVersionConcurrencyControl.WriteEntry we = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
index 42abeae..7161e1e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
@@ -19,108 +19,42 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
-import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
-
 import java.io.IOException;
+import java.io.OutputStream;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
 /**
  * Writer for protobuf-based WAL.
  */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class ProtobufLogWriter extends WriterBase {
-  private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class);
-  protected FSDataOutputStream output;
-  protected Codec.Encoder cellEncoder;
-  protected WALCellCodec.ByteStringCompressor compressor;
-  private boolean trailerWritten;
-  private WALTrailer trailer;
-  // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
-  // than this size, it is written/read respectively, with a WARN message in the log.
-  private int trailerWarnSize;
-
-  public ProtobufLogWriter() {
-    super();
-  }
-
-  protected WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
-      throws IOException {
-    return WALCellCodec.create(conf, null, compressionContext);
-  }
-
-  protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
-      throws IOException {
-    if (!builder.hasWriterClsName()) {
-      builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
-    }
-    if (!builder.hasCellCodecClsName()) {
-      builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
-    }
-    return builder.build();
-  }
+@InterfaceAudience.Private
+public class ProtobufLogWriter extends AbstractProtobufLogWriter
+    implements DefaultWALProvider.Writer {
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
-  throws IOException {
-    super.init(fs, path, conf, overwritable);
-    assert this.output == null;
-    boolean doCompress = initializeCompressionContext(conf, path);
-    this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
-    int bufferSize = FSUtils.getDefaultBufferSize(fs);
-    short replication = (short)conf.getInt(
-        "hbase.regionserver.hlog.replication", FSUtils.getDefaultReplication(fs, path));
-    long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
-        FSUtils.getDefaultBlockSize(fs, path));
-    output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize, null);
-    output.write(ProtobufLogReader.PB_WAL_MAGIC);
-    boolean doTagCompress = doCompress
-        && conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
-    buildWALHeader(conf,
-        WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))
-        .writeDelimitedTo(output);
-
-    initAfterHeader(doCompress);
-
-    // instantiate trailer to default value.
-    trailer = WALTrailer.newBuilder().build();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
-    }
-  }
+  private static final Log LOG = LogFactory.getLog(ProtobufLogWriter.class);
 
-  protected void initAfterHeader(boolean doCompress) throws IOException {
-    WALCellCodec codec = getCodec(conf, this.compressionContext);
-    this.cellEncoder = codec.getEncoder(this.output);
-    if (doCompress) {
-      this.compressor = codec.getByteStringCompressor();
-    }
-  }
+  protected FSDataOutputStream output;
 
   @Override
   public void append(Entry entry) throws IOException {
     entry.setCompressionContext(compressionContext);
-    entry.getKey().getBuilder(compressor).
-      setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output);
+    entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
+        .writeDelimitedTo(output);
     for (Cell cell : entry.getEdit().getCells()) {
       // cellEncoder must assume little about the stream, since we write PB and cells in turn.
       cellEncoder.write(cell);
     }
+    length.set(output.getPos());
   }
 
   @Override
@@ -137,32 +71,6 @@ public class ProtobufLogWriter extends WriterBase {
     }
   }
 
-  WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
-    return builder.build();
-  }
-
-  private void writeWALTrailer() {
-    try {
-      int trailerSize = 0;
-      if (this.trailer == null) {
-        // use default trailer.
-        LOG.warn("WALTrailer is null. Continuing with default.");
-        this.trailer = buildWALTrailer(WALTrailer.newBuilder());
-        trailerSize = this.trailer.getSerializedSize();
-      } else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
-        // continue writing after warning the user.
-        LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " +
-          trailerSize + " > " + this.trailerWarnSize);
-      }
-      this.trailer.writeTo(output);
-      output.writeInt(trailerSize);
-      output.write(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC);
-      this.trailerWritten = true;
-    } catch (IOException ioe) {
-      LOG.warn("Failed to write trailer, non-fatal, continuing...", ioe);
-    }
-  }
-
   @Override
   public void sync() throws IOException {
     FSDataOutputStream fsdos = this.output;
@@ -171,21 +79,35 @@ public class ProtobufLogWriter extends WriterBase {
     fsdos.hflush();
   }
 
+  public FSDataOutputStream getStream() {
+    return this.output;
+  }
+
+  @SuppressWarnings("deprecation")
   @Override
-  public long getLength() throws IOException {
-    try {
-      return this.output.getPos();
-    } catch (NullPointerException npe) {
-      // Concurrent close...
-      throw new IOException(npe);
-    }
+  protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
+      short replication, long blockSize) throws IOException {
+    this.output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize,
+      null);
   }
 
-  public FSDataOutputStream getStream() {
+  @Override
+  protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
+    output.write(magic);
+    header.writeDelimitedTo(output);
+    return output.getPos();
+  }
+
+  @Override
+  protected OutputStream getOutputStreamForCellEncoder() {
     return this.output;
   }
 
-  void setWALTrailer(WALTrailer walTrailer) {
-    this.trailer = walTrailer;
+  @Override
+  protected long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic) throws IOException {
+    trailer.writeTo(output);
+    output.writeInt(trailer.getSerializedSize());
+    output.write(magic);
+    return output.getPos();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
index 7de8367..b5c9a2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
@@ -18,33 +18,28 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.htrace.Span;
 
 /**
- * A Future on a filesystem sync call.  It given to a client or 'Handler' for it to wait on till
- * the sync completes.
- *
- * <p>Handlers coming in call append, append, append, and then do a flush/sync of
- * the edits they have appended the WAL before returning. Since sync takes a while to
- * complete, we give the Handlers back this sync future to wait on until the
- * actual HDFS sync completes. Meantime this sync future goes across the ringbuffer and into a
- * sync runner thread; when it completes, it finishes up the future, the handler get or failed
- * check completes and the Handler can then progress.
+ * A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till the
+ * sync completes.
+ * <p>
+ * Handlers coming in call append, append, append, and then do a flush/sync of the edits they have
+ * appended the WAL before returning. Since sync takes a while to complete, we give the Handlers
+ * back this sync future to wait on until the actual HDFS sync completes. Meantime this sync future
+ * goes across a queue and is handled by a background thread; when it completes, it finishes up the
+ * future, the handler get or failed check completes and the Handler can then progress.
  * <p>
- * This is just a partial implementation of Future; we just implement get and
- * failure.  Unimplemented methods throw {@link UnsupportedOperationException}.
+ * This is just a partial implementation of Future; we just implement get and failure.
  * <p>
- * There is not a one-to-one correlation between dfs sync invocations and
- * instances of this class. A single dfs sync call may complete and mark many
- * SyncFutures as done; i.e. we batch up sync calls rather than do a dfs sync
- * call every time a Handler asks for it.
+ * There is not a one-to-one correlation between dfs sync invocations and instances of this class. A
+ * single dfs sync call may complete and mark many SyncFutures as done; i.e. we batch up sync calls
+ * rather than do a dfs sync call every time a Handler asks for it.
  * <p>
- * SyncFutures are immutable but recycled. Call #reset(long, Span) before use even
- * if it the first time, start the sync, then park the 'hitched' thread on a call to
- * #get().
+ * SyncFutures are immutable but recycled. Call #reset(long, Span) before use even if it the first
+ * time, start the sync, then park the 'hitched' thread on a call to #get().
  */
 @InterfaceAudience.Private
 class SyncFuture {
@@ -54,17 +49,17 @@ class SyncFuture {
   private static final long NOT_DONE = 0;
 
   /**
-   * The sequence at which we were added to the ring buffer.
+   * The transaction id of this operation, monotonically increases.
    */
-  private long ringBufferSequence;
+  private long txid;
 
   /**
-   * The sequence that was set in here when we were marked done. Should be equal
-   * or > ringBufferSequence.  Put this data member into the NOT_DONE state while this
-   * class is in use.  But for the first position on construction, let it be -1 so we can
-   * immediately call {@link #reset(long, Span)} below and it will work.
+   * The transaction id that was set in here when we were marked done. Should be equal or > txnId.
+   * Put this data member into the NOT_DONE state while this class is in use. But for the first
+   * position on construction, let it be -1 so we can immediately call {@link #reset(long, Span)}
+   * below and it will work.
    */
-  private long doneSequence = -1;
+  private long doneTxid = -1;
 
   /**
    * If error, the associated throwable. Set when the future is 'done'.
@@ -79,80 +74,83 @@ class SyncFuture {
   private Span span;
 
   /**
-   * Call this method to clear old usage and get it ready for new deploy. Call
-   * this method even if it is being used for the first time.
-   *
-   * @param sequence sequenceId from this Future's position in the RingBuffer
+   * Call this method to clear old usage and get it ready for new deploy. Call this method even if
+   * it is being used for the first time.
+   * @param txnId the new transaction id
    * @return this
    */
-  synchronized SyncFuture reset(final long sequence) {
-    return reset(sequence, null);
+  synchronized SyncFuture reset(final long txnId) {
+    return reset(txnId, null);
   }
 
   /**
-   * Call this method to clear old usage and get it ready for new deploy. Call
-   * this method even if it is being used for the first time.
-   *
+   * Call this method to clear old usage and get it ready for new deploy. Call this method even if
+   * it is being used for the first time.
    * @param sequence sequenceId from this Future's position in the RingBuffer
-   * @param span curren span, detached from caller. Don't forget to attach it when
-   *             resuming after a call to {@link #get()}.
+   * @param span curren span, detached from caller. Don't forget to attach it when resuming after a
+   *          call to {@link #get()}.
    * @return this
    */
-  synchronized SyncFuture reset(final long sequence, Span span) {
-    if (t != null && t != Thread.currentThread()) throw new IllegalStateException();
+  synchronized SyncFuture reset(final long txnId, Span span) {
+    if (t != null && t != Thread.currentThread()) {
+      throw new IllegalStateException();
+    }
     t = Thread.currentThread();
-    if (!isDone()) throw new IllegalStateException("" + sequence + " " + Thread.currentThread());
-    this.doneSequence = NOT_DONE;
-    this.ringBufferSequence = sequence;
+    if (!isDone()) {
+      throw new IllegalStateException("" + txnId + " " + Thread.currentThread());
+    }
+    this.doneTxid = NOT_DONE;
+    this.txid = txnId;
     this.span = span;
     return this;
   }
 
   @Override
   public synchronized String toString() {
-    return "done=" + isDone() + ", ringBufferSequence=" + this.ringBufferSequence;
+    return "done=" + isDone() + ", txid=" + this.txid;
   }
 
-  synchronized long getRingBufferSequence() {
-    return this.ringBufferSequence;
+  synchronized long getTxid() {
+    return this.txid;
   }
 
   /**
-   * Retrieve the {@code span} instance from this Future. EventHandler calls
-   * this method to continue the span. Thread waiting on this Future musn't call
-   * this method until AFTER calling {@link #get()} and the future has been
-   * released back to the originating thread.
+   * Retrieve the {@code span} instance from this Future. EventHandler calls this method to continue
+   * the span. Thread waiting on this Future musn't call this method until AFTER calling
+   * {@link #get()} and the future has been released back to the originating thread.
    */
   synchronized Span getSpan() {
     return this.span;
   }
 
   /**
-   * Used to re-attach a {@code span} to the Future. Called by the EventHandler
-   * after a it has completed processing and detached the span from its scope.
+   * Used to re-attach a {@code span} to the Future. Called by the EventHandler after a it has
+   * completed processing and detached the span from its scope.
    */
   synchronized void setSpan(Span span) {
     this.span = span;
   }
 
   /**
-   * @param sequence Sync sequence at which this future 'completed'.
-   * @param t Can be null.  Set if we are 'completing' on error (and this 't' is the error).
-   * @return True if we successfully marked this outstanding future as completed/done.
-   * Returns false if this future is already 'done' when this method called.
+   * @param txid the transaction id at which this future 'completed'.
+   * @param t Can be null. Set if we are 'completing' on error (and this 't' is the error).
+   * @return True if we successfully marked this outstanding future as completed/done. Returns false
+   *         if this future is already 'done' when this method called.
    */
-  synchronized boolean done(final long sequence, final Throwable t) {
-    if (isDone()) return false;
+  synchronized boolean done(final long txid, final Throwable t) {
+    if (isDone()) {
+      return false;
+    }
     this.throwable = t;
-    if (sequence < this.ringBufferSequence) {
+    if (txid < this.txid) {
       // Something badly wrong.
       if (throwable == null) {
-        this.throwable = new IllegalStateException("sequence=" + sequence +
-          ", ringBufferSequence=" + this.ringBufferSequence);
+        this.throwable =
+            new IllegalStateException("done txid=" + txid + ", my txid=" + this.txid);
       }
     }
     // Mark done.
-    this.doneSequence = sequence;
+    this.doneTxid = txid;
     // Wake up waiting threads.
     notify();
     return true;
@@ -166,21 +164,14 @@ class SyncFuture {
     while (!isDone()) {
       wait(1000);
     }
-    if (this.throwable != null) throw new ExecutionException(this.throwable);
-    return this.doneSequence;
-  }
-
-  public Long get(long timeout, TimeUnit unit)
-  throws InterruptedException, ExecutionException {
-    throw new UnsupportedOperationException();
-  }
-
-  public boolean isCancelled() {
-    throw new UnsupportedOperationException();
+    if (this.throwable != null) {
+      throw new ExecutionException(this.throwable);
+    }
+    return this.doneTxid;
   }
 
   synchronized boolean isDone() {
-    return this.doneSequence != NOT_DONE;
+    return this.doneTxid != NOT_DONE;
   }
 
   synchronized boolean isThrowable() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java
deleted file mode 100644
index 8188e02..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WriterBase.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver.wal;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.io.util.LRUDictionary;
-import org.apache.hadoop.hbase.util.FSUtils;
-
-import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-
-/**
- * Context used by our wal dictionary compressor. Null if we're not to do our
- * custom dictionary compression.
- */
-@InterfaceAudience.Private
-public abstract class WriterBase implements DefaultWALProvider.Writer {
-
-  protected CompressionContext compressionContext;
-  protected Configuration conf;
-
-  @Override
-  public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable) throws IOException {
-    this.conf = conf;
-  }
-
-  public boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
-    boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
-    if (doCompress) {
-      try {
-        this.compressionContext = new CompressionContext(LRUDictionary.class,
-            FSUtils.isRecoveredEdits(path), conf.getBoolean(
-                CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
-      } catch (Exception e) {
-        throw new IOException("Failed to initiate CompressionContext", e);
-      }
-    }
-    return doCompress;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
index ea71701..32fe48b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -417,7 +418,6 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     public boolean progress() {
       return DFS_CLIENT_ADAPTOR.isClientRunning(client);
     }
-
   }
 
   static {
@@ -579,6 +579,18 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     return futureList;
   }
 
+  /**
+   * Exception other than RemoteException thrown when calling create on namenode
+   */
+  public static class NameNodeException extends IOException {
+
+    private static final long serialVersionUID = 3143237406477095390L;
+
+    public NameNodeException(Throwable cause) {
+      super(cause);
+    }
+  }
+
   private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,
       boolean overwrite, boolean createParent, short replication, long blockSize,
       EventLoop eventLoop) throws IOException {
@@ -587,11 +599,20 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     DFSClient client = dfs.getClient();
     String clientName = client.getClientName();
     ClientProtocol namenode = client.getNamenode();
-    HdfsFileStatus stat = FILE_CREATER.create(namenode, src,
-      FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
-      new EnumSetWritable<CreateFlag>(
-          overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
-      createParent, replication, blockSize);
+    HdfsFileStatus stat;
+    try {
+      stat = FILE_CREATER.create(namenode, src,
+        FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
+        new EnumSetWritable<CreateFlag>(
+            overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
+        createParent, replication, blockSize);
+    } catch (Exception e) {
+      if (e instanceof RemoteException) {
+        throw (RemoteException) e;
+      } else {
+        throw new NameNodeException(e);
+      }
+    }
     beginFileLease(client, src, stat.getFileId());
     boolean succ = false;
     LocatedBlock locatedBlock = null;
@@ -656,6 +677,13 @@ public class FanOutOneBlockAsyncDFSOutputHelper {
     }.resolve(dfs, f);
   }
 
+  public static boolean shouldRetryCreate(RemoteException e) {
+    // RetryStartFileException is introduced in HDFS 2.6+, so here we can only use the class name.
+    // For exceptions other than this, we just throw it out. This is same with
+    // DFSOutputStream.newStreamForCreate.
+    return e.getClassName().endsWith("RetryStartFileException");
+  }
+
   static void completeFile(DFSClient client, ClientProtocol namenode, String src, String clientName,
       ExtendedBlock block, long fileId) {
     for (int retry = 0;; retry++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
new file mode 100644
index 0000000..2f5c299
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.wal;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Base class of a WAL Provider that returns a single thread safe WAL that writes to HDFS. By
+ * default, this implementation picks a directory in HDFS based on a combination of
+ * <ul>
+ * <li>the HBase root directory
+ * <li>HConstants.HREGION_LOGDIR_NAME
+ * <li>the given factory's factoryId (usually identifying the regionserver by host:port)
+ * </ul>
+ * It also uses the providerId to differentiate among files.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implements WALProvider {
+
+  private static final Log LOG = LogFactory.getLog(AbstractFSWALProvider.class);
+
+  // Only public so classes back in regionserver.wal can access
+  public interface Reader extends WAL.Reader {
+    /**
+     * @param fs File system.
+     * @param path Path.
+     * @param c Configuration.
+     * @param s Input stream that may have been pre-opened by the caller; may be null.
+     */
+    void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
+  }
+
+  protected volatile T wal;
+  protected WALFactory factory = null;
+  protected Configuration conf = null;
+  protected List<WALActionsListener> listeners = null;
+  protected String providerId = null;
+  protected AtomicBoolean initialized = new AtomicBoolean(false);
+  // for default wal provider, logPrefix won't change
+  protected String logPrefix = null;
+
+  /**
+   * we synchronized on walCreateLock to prevent wal recreation in different threads
+   */
+  private final Object walCreateLock = new Object();
+
+  /**
+   * @param factory factory that made us, identity used for FS layout. may not be null
+   * @param conf may not be null
+   * @param listeners may be null
+   * @param providerId differentiate between providers from one facotry, used for FS layout. may be
+   *          null
+   */
+  @Override
+  public void init(WALFactory factory, Configuration conf, List<WALActionsListener> listeners,
+      String providerId) throws IOException {
+    if (!initialized.compareAndSet(false, true)) {
+      throw new IllegalStateException("WALProvider.init should only be called once.");
+    }
+    this.factory = factory;
+    this.conf = conf;
+    this.listeners = listeners;
+    this.providerId = providerId;
+    // get log prefix
+    StringBuilder sb = new StringBuilder().append(factory.factoryId);
+    if (providerId != null) {
+      if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
+        sb.append(providerId);
+      } else {
+        sb.append(WAL_FILE_NAME_DELIMITER).append(providerId);
+      }
+    }
+    logPrefix = sb.toString();
+    doInit(conf);
+  }
+
+  @Override
+  public WAL getWAL(byte[] identifier, byte[] namespace) throws IOException {
+    T walCopy = wal;
+    if (walCopy == null) {
+      // only lock when need to create wal, and need to lock since
+      // creating hlog on fs is time consuming
+      synchronized (walCreateLock) {
+        walCopy = wal;
+        if (walCopy == null) {
+          walCopy = createWAL();
+          wal = walCopy;
+        }
+      }
+    }
+    return walCopy;
+  }
+
+  protected abstract T createWAL() throws IOException;
+
+  protected abstract void doInit(Configuration conf) throws IOException;
+
+  @Override
+  public void shutdown() throws IOException {
+    T log = this.wal;
+    if (log != null) {
+      log.shutdown();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    T log = this.wal;
+    if (log != null) {
+      log.close();
+    }
+  }
+
+  /**
+   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta, count the
+   * number of files (rolled and active). if either of them aren't, count 0 for that provider.
+   */
+  @Override
+  public long getNumLogFiles() {
+    T log = this.wal;
+    return log == null ? 0 : log.getNumLogFiles();
+  }
+
+  /**
+   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta, count the
+   * size of files (rolled and active). if either of them aren't, count 0 for that provider.
+   */
+  @Override
+  public long getLogFileSize() {
+    T log = this.wal;
+    return log == null ? 0 : log.getLogFileSize();
+  }
+
+  /**
+   * returns the number of rolled WAL files.
+   */
+  @VisibleForTesting
+  public static int getNumRolledLogFiles(WAL wal) {
+    return ((AbstractFSWAL<?>) wal).getNumRolledLogFiles();
+  }
+
+  /**
+   * return the current filename from the current wal.
+   */
+  @VisibleForTesting
+  public static Path getCurrentFileName(final WAL wal) {
+    return ((AbstractFSWAL<?>) wal).getCurrentFileName();
+  }
+
+  /**
+   * request a log roll, but don't actually do it.
+   */
+  @VisibleForTesting
+  static void requestLogRoll(final WAL wal) {
+    ((AbstractFSWAL<?>) wal).requestLogRoll();
+  }
+
+  // should be package private; more visible for use in AbstractFSWAL
+  public static final String WAL_FILE_NAME_DELIMITER = ".";
+  /** The hbase:meta region's WAL filename extension */
+  @VisibleForTesting
+  public static final String META_WAL_PROVIDER_ID = ".meta";
+  static final String DEFAULT_PROVIDER_ID = "default";
+
+  // Implementation details that currently leak in tests or elsewhere follow
+  /** File Extension used while splitting an WAL into regions (HBASE-2312) */
+  public static final String SPLITTING_EXT = "-splitting";
+
+  /**
+   * It returns the file create timestamp from the file name. For name format see
+   * {@link #validateWALFilename(String)} public until remaining tests move to o.a.h.h.wal
+   * @param wal must not be null
+   * @return the file number that is part of the WAL file name
+   */
+  @VisibleForTesting
+  public static long extractFileNumFromWAL(final WAL wal) {
+    final Path walName = ((AbstractFSWAL<?>) wal).getCurrentFileName();
+    if (walName == null) {
+      throw new IllegalArgumentException("The WAL path couldn't be null");
+    }
+    final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
+    return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2 : 1)]);
+  }
+
+  /**
+   * Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for
+   * description.
+   */
+  private static final Pattern pattern = Pattern
+      .compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*");
+
+  /**
+   * A WAL file name is of the format: &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}
+   * &lt;file-creation-timestamp&gt;[.meta]. provider-name is usually made up of a server-name and a
+   * provider-id
+   * @param filename name of the file to validate
+   * @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt> otherwise
+   */
+  public static boolean validateWALFilename(String filename) {
+    return pattern.matcher(filename).matches();
+  }
+
+  /**
+   * Construct the directory name for all WALs on a given server.
+   * @param serverName Server name formatted as described in {@link ServerName}
+   * @return the relative WAL directory name, e.g. <code>.logs/1.example.org,60030,12345</code> if
+   *         <code>serverName</code> passed is <code>1.example.org,60030,12345</code>
+   */
+  public static String getWALDirectoryName(final String serverName) {
+    StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
+    dirName.append("/");
+    dirName.append(serverName);
+    return dirName.toString();
+  }
+
+  /**
+   * Pulls a ServerName out of a Path generated according to our layout rules. In the below layouts,
+   * this method ignores the format of the logfile component. Current format: [base directory for
+   * hbase]/hbase/.logs/ServerName/logfile or [base directory for
+   * hbase]/hbase/.logs/ServerName-splitting/logfile Expected to work for individual log files and
+   * server-specific directories.
+   * @return null if it's not a log file. Returns the ServerName of the region server that created
+   *         this log file otherwise.
+   */
+  public static ServerName getServerNameFromWALDirectoryName(Configuration conf, String path)
+      throws IOException {
+    if (path == null || path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
+      return null;
+    }
+
+    if (conf == null) {
+      throw new IllegalArgumentException("parameter conf must be set");
+    }
+
+    final String rootDir = conf.get(HConstants.HBASE_DIR);
+    if (rootDir == null || rootDir.isEmpty()) {
+      throw new IllegalArgumentException(HConstants.HBASE_DIR + " key not found in conf.");
+    }
+
+    final StringBuilder startPathSB = new StringBuilder(rootDir);
+    if (!rootDir.endsWith("/")) {
+      startPathSB.append('/');
+    }
+    startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
+    if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/")) {
+      startPathSB.append('/');
+    }
+    final String startPath = startPathSB.toString();
+
+    String fullPath;
+    try {
+      fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
+    } catch (IllegalArgumentException e) {
+      LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
+      return null;
+    }
+
+    if (!fullPath.startsWith(startPath)) {
+      return null;
+    }
+
+    final String serverNameAndFile = fullPath.substring(startPath.length());
+
+    if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
+      // Either it's a file (not a directory) or it's not a ServerName format
+      return null;
+    }
+
+    Path p = new Path(path);
+    return getServerNameFromWALDirectoryName(p);
+  }
+
+  /**
+   * This function returns region server name from a log file name which is in one of the following
+   * formats:
+   * <ul>
+   * <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;-splitting/...</li>
+   * <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;/...</li>
+   * </ul>
+   * @return null if the passed in logFile isn't a valid WAL file path
+   */
+  public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
+    String logDirName = logFile.getParent().getName();
+    // We were passed the directory and not a file in it.
+    if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
+      logDirName = logFile.getName();
+    }
+    ServerName serverName = null;
+    if (logDirName.endsWith(SPLITTING_EXT)) {
+      logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
+    }
+    try {
+      serverName = ServerName.parseServerName(logDirName);
+    } catch (IllegalArgumentException ex) {
+      serverName = null;
+      LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
+    }
+    if (serverName != null && serverName.getStartcode() < 0) {
+      LOG.warn("Invalid log file path=" + logFile);
+      serverName = null;
+    }
+    return serverName;
+  }
+
+  public static boolean isMetaFile(Path p) {
+    return isMetaFile(p.getName());
+  }
+
+  public static boolean isMetaFile(String p) {
+    if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Get prefix of the log from its name, assuming WAL name in format of
+   * log_prefix.filenumber.log_suffix
+   * @param name Name of the WAL to parse
+   * @return prefix of the log
+   * @see AbstractFSWAL#getCurrentFileName()
+   */
+  public static String getWALPrefixFromWALName(String name) {
+    int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
+    return name.substring(0, endIndex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
new file mode 100644
index 0000000..bc142ce
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.wal;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
+
+import io.netty.channel.EventLoop;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+/**
+ * A WAL provider that use {@link AsyncFSWAL}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
+
+  // Only public so classes back in regionserver.wal can access
+  public interface AsyncWriter extends WALProvider.AsyncWriter {
+    void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
+  }
+
+  private EventLoopGroup eventLoopGroup = null;
+
+  @Override
+  protected AsyncFSWAL createWAL() throws IOException {
+    return new AsyncFSWAL(FileSystem.get(conf), FSUtils.getRootDir(conf),
+        getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
+        true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null,
+        eventLoopGroup.next());
+  }
+
+  @Override
+  protected void doInit(Configuration conf) throws IOException {
+    eventLoopGroup = new NioEventLoopGroup(1, Threads.newDaemonThreadFactory("AsyncFSWAL"));
+  }
+
+  /**
+   * public because of AsyncFSWAL. Should be package-private
+   */
+  public static AsyncWriter createAsyncWriter(Configuration conf, FileSystem fs, Path path,
+      boolean overwritable, EventLoop eventLoop) throws IOException {
+    AsyncWriter writer = new AsyncProtobufLogWriter(eventLoop);
+    writer.init(fs, path, conf, overwritable);
+    return writer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
index 027e7a2..9f0d0ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
@@ -19,357 +19,42 @@
 package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.regex.Pattern;
-
-import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.FSUtils;
-
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 // imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
- * A WAL Provider that returns a single thread safe WAL that writes to HDFS.
- * By default, this implementation picks a directory in HDFS based on a combination of
- * <ul>
- *   <li>the HBase root directory
- *   <li>HConstants.HREGION_LOGDIR_NAME
- *   <li>the given factory's factoryId (usually identifying the regionserver by host:port)
- * </ul>
- * It also uses the providerId to diffentiate among files.
- *
+ * A WAL provider that use {@link FSHLog}.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class DefaultWALProvider implements WALProvider {
-  private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
+public class DefaultWALProvider extends AbstractFSWALProvider<FSHLog> {
 
-  // Only public so classes back in regionserver.wal can access
-  public interface Reader extends WAL.Reader {
-    /**
-     * @param fs File system.
-     * @param path Path.
-     * @param c Configuration.
-     * @param s Input stream that may have been pre-opened by the caller; may be null.
-     */
-    void init(FileSystem fs, Path path, Configuration c, FSDataInputStream s) throws IOException;
-  }
+  private static final Log LOG = LogFactory.getLog(DefaultWALProvider.class);
 
   // Only public so classes back in regionserver.wal can access
   public interface Writer extends WALProvider.Writer {
     void init(FileSystem fs, Path path, Configuration c, boolean overwritable) throws IOException;
   }
 
-  protected volatile FSHLog log = null;
-  private WALFactory factory = null;
-  private Configuration conf = null;
-  private List<WALActionsListener> listeners = null;
-  private String providerId = null;
-  private AtomicBoolean initialized = new AtomicBoolean(false);
-  // for default wal provider, logPrefix won't change
-  private String logPrefix = null;
-
-  /**
-   * we synchronized on walCreateLock to prevent wal recreation in different threads
-   */
-  private final Object walCreateLock = new Object();
-
-  /**
-   * @param factory factory that made us, identity used for FS layout. may not be null
-   * @param conf may not be null
-   * @param listeners may be null
-   * @param providerId differentiate between providers from one facotry, used for FS layout. may be
-   *                   null
-   */
-  @Override
-  public void init(final WALFactory factory, final Configuration conf,
-      final List<WALActionsListener> listeners, String providerId) throws IOException {
-    if (!initialized.compareAndSet(false, true)) {
-      throw new IllegalStateException("WALProvider.init should only be called once.");
-    }
-    this.factory = factory;
-    this.conf = conf;
-    this.listeners = listeners;
-    this.providerId = providerId;
-    // get log prefix
-    StringBuilder sb = new StringBuilder().append(factory.factoryId);
-    if (providerId != null) {
-      if (providerId.startsWith(WAL_FILE_NAME_DELIMITER)) {
-        sb.append(providerId);
-      } else {
-        sb.append(WAL_FILE_NAME_DELIMITER).append(providerId);
-      }
-    }
-    logPrefix = sb.toString();
-  }
-
-  @Override
-  public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException {
-    if (log == null) {
-      // only lock when need to create wal, and need to lock since
-      // creating hlog on fs is time consuming
-      synchronized (walCreateLock) {
-        if (log == null) {
-          log = new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
-              getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf,
-              listeners, true, logPrefix,
-              META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
-        }
-      }
-    }
-    return log;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (log != null) log.close();
-  }
-
-  @Override
-  public void shutdown() throws IOException {
-    if (log != null) log.shutdown();
-  }
-
-  // should be package private; more visible for use in FSHLog
-  public static final String WAL_FILE_NAME_DELIMITER = ".";
-  /** The hbase:meta region's WAL filename extension */
-  @VisibleForTesting
-  public static final String META_WAL_PROVIDER_ID = ".meta";
-  static final String DEFAULT_PROVIDER_ID = "default";
-
-  // Implementation details that currently leak in tests or elsewhere follow
-  /** File Extension used while splitting an WAL into regions (HBASE-2312) */
-  public static final String SPLITTING_EXT = "-splitting";
-
-  /**
-   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
-   * count the number of files (rolled and active). if either of them aren't, count 0
-   * for that provider.
-   */
-  @Override
-  public long getNumLogFiles() {
-    return log == null ? 0 : this.log.getNumLogFiles();
-  }
-
-  /**
-   * iff the given WALFactory is using the DefaultWALProvider for meta and/or non-meta,
-   * count the size of files (rolled and active). if either of them aren't, count 0
-   * for that provider.
-   */
-  @Override
-  public long getLogFileSize() {
-    return log == null ? 0 : this.log.getLogFileSize();
-  }
-
-  /**
-   * returns the number of rolled WAL files.
-   */
-  @VisibleForTesting
-  public static int getNumRolledLogFiles(WAL wal) {
-    return ((FSHLog)wal).getNumRolledLogFiles();
-  }
-
-  /**
-   * return the current filename from the current wal.
-   */
-  @VisibleForTesting
-  public static Path getCurrentFileName(final WAL wal) {
-    return ((FSHLog)wal).getCurrentFileName();
-  }
-
-  /**
-   * request a log roll, but don't actually do it.
-   */
-  @VisibleForTesting
-  static void requestLogRoll(final WAL wal) {
-    ((FSHLog)wal).requestLogRoll();
-  }
-
-  /**
-   * It returns the file create timestamp from the file name.
-   * For name format see {@link #validateWALFilename(String)}
-   * public until remaining tests move to o.a.h.h.wal
-   * @param wal must not be null
-   * @return the file number that is part of the WAL file name
-   */
-  @VisibleForTesting
-  public static long extractFileNumFromWAL(final WAL wal) {
-    final Path walName = ((FSHLog)wal).getCurrentFileName();
-    if (walName == null) {
-      throw new IllegalArgumentException("The WAL path couldn't be null");
-    }
-    final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
-    return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
-  }
-
-  /**
-   * Pattern used to validate a WAL file name
-   * see {@link #validateWALFilename(String)} for description.
-   */
-  private static final Pattern pattern = Pattern.compile(".*\\.\\d*("+META_WAL_PROVIDER_ID+")*");
-
-  /**
-   * A WAL file name is of the format:
-   * &lt;wal-name&gt;{@link #WAL_FILE_NAME_DELIMITER}&lt;file-creation-timestamp&gt;[.meta].
-   *
-   * provider-name is usually made up of a server-name and a provider-id
-   *
-   * @param filename name of the file to validate
-   * @return <tt>true</tt> if the filename matches an WAL, <tt>false</tt>
-   *         otherwise
-   */
-  public static boolean validateWALFilename(String filename) {
-    return pattern.matcher(filename).matches();
-  }
-
-  /**
-   * Construct the directory name for all WALs on a given server.
-   *
-   * @param serverName
-   *          Server name formatted as described in {@link ServerName}
-   * @return the relative WAL directory name, e.g.
-   *         <code>.logs/1.example.org,60030,12345</code> if
-   *         <code>serverName</code> passed is
-   *         <code>1.example.org,60030,12345</code>
-   */
-  public static String getWALDirectoryName(final String serverName) {
-    StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
-    dirName.append("/");
-    dirName.append(serverName);
-    return dirName.toString();
-  }
-
-  /**
-   * Pulls a ServerName out of a Path generated according to our layout rules.
-   *
-   * In the below layouts, this method ignores the format of the logfile component.
-   *
-   * Current format:
-   *
-   * [base directory for hbase]/hbase/.logs/ServerName/logfile
-   *      or
-   * [base directory for hbase]/hbase/.logs/ServerName-splitting/logfile
-   *
-   * Expected to work for individual log files and server-specific directories.
-   *
-   * @return null if it's not a log file. Returns the ServerName of the region
-   *         server that created this log file otherwise.
-   */
-  public static ServerName getServerNameFromWALDirectoryName(Configuration conf, String path)
-      throws IOException {
-    if (path == null
-        || path.length() <= HConstants.HREGION_LOGDIR_NAME.length()) {
-      return null;
-    }
-
-    if (conf == null) {
-      throw new IllegalArgumentException("parameter conf must be set");
-    }
-
-    final String rootDir = conf.get(HConstants.HBASE_DIR);
-    if (rootDir == null || rootDir.isEmpty()) {
-      throw new IllegalArgumentException(HConstants.HBASE_DIR
-          + " key not found in conf.");
-    }
-
-    final StringBuilder startPathSB = new StringBuilder(rootDir);
-    if (!rootDir.endsWith("/"))
-      startPathSB.append('/');
-    startPathSB.append(HConstants.HREGION_LOGDIR_NAME);
-    if (!HConstants.HREGION_LOGDIR_NAME.endsWith("/"))
-      startPathSB.append('/');
-    final String startPath = startPathSB.toString();
-
-    String fullPath;
-    try {
-      fullPath = FileSystem.get(conf).makeQualified(new Path(path)).toString();
-    } catch (IllegalArgumentException e) {
-      LOG.info("Call to makeQualified failed on " + path + " " + e.getMessage());
-      return null;
-    }
-
-    if (!fullPath.startsWith(startPath)) {
-      return null;
-    }
-
-    final String serverNameAndFile = fullPath.substring(startPath.length());
-
-    if (serverNameAndFile.indexOf('/') < "a,0,0".length()) {
-      // Either it's a file (not a directory) or it's not a ServerName format
-      return null;
-    }
-
-    Path p = new Path(path);
-    return getServerNameFromWALDirectoryName(p);
-  }
-
-  /**
-   * This function returns region server name from a log file name which is in one of the following
-   * formats:
-   * <ul>
-   *   <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;-splitting/...</li>
-   *   <li>hdfs://&lt;name node&gt;/hbase/.logs/&lt;server name&gt;/...</li>
-   * </ul>
-   * @param logFile
-   * @return null if the passed in logFile isn't a valid WAL file path
-   */
-  public static ServerName getServerNameFromWALDirectoryName(Path logFile) {
-    String logDirName = logFile.getParent().getName();
-    // We were passed the directory and not a file in it.
-    if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
-      logDirName = logFile.getName();
-    }
-    ServerName serverName = null;
-    if (logDirName.endsWith(SPLITTING_EXT)) {
-      logDirName = logDirName.substring(0, logDirName.length() - SPLITTING_EXT.length());
-    }
-    try {
-      serverName = ServerName.parseServerName(logDirName);
-    } catch (IllegalArgumentException ex) {
-      serverName = null;
-      LOG.warn("Cannot parse a server name from path=" + logFile + "; " + ex.getMessage());
-    }
-    if (serverName != null && serverName.getStartcode() < 0) {
-      LOG.warn("Invalid log file path=" + logFile);
-      serverName = null;
-    }
-    return serverName;
-  }
-
-  public static boolean isMetaFile(Path p) {
-    return isMetaFile(p.getName());
-  }
-
-  public static boolean isMetaFile(String p) {
-    if (p != null && p.endsWith(META_WAL_PROVIDER_ID)) {
-      return true;
-    }
-    return false;
-  }
-
   /**
    * public because of FSHLog. Should be package-private
    */
   public static Writer createWriter(final Configuration conf, final FileSystem fs, final Path path,
-      final boolean overwritable)
-      throws IOException {
+      final boolean overwritable) throws IOException {
     // Configuration already does caching for the Class lookup.
     Class<? extends Writer> logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
-        ProtobufLogWriter.class, Writer.class);
+      ProtobufLogWriter.class, Writer.class);
     try {
       Writer writer = logWriterClass.newInstance();
       writer.init(fs, path, conf, overwritable);
@@ -380,15 +65,14 @@ public class DefaultWALProvider implements WALProvider {
     }
   }
 
-  /**
-   * Get prefix of the log from its name, assuming WAL name in format of
-   * log_prefix.filenumber.log_suffix @see {@link FSHLog#getCurrentFileName()}
-   * @param name Name of the WAL to parse
-   * @return prefix of the log
-   */
-  public static String getWALPrefixFromWALName(String name) {
-    int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
-    return name.substring(0, endIndex);
+  @Override
+  protected FSHLog createWAL() throws IOException {
+    return new FSHLog(FileSystem.get(conf), FSUtils.getRootDir(conf),
+        getWALDirectoryName(factory.factoryId), HConstants.HREGION_OLDLOGDIR_NAME, conf, listeners,
+        true, logPrefix, META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
   }
 
+  @Override
+  protected void doInit(Configuration conf) throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index c3d4b2c..028c60b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -215,6 +215,10 @@ class DisabledWALProvider implements WALProvider {
     public String toString() {
       return "WAL disabled.";
     }
+
+    @Override
+    public void logRollerExited() {
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index 0b83528..051ce54 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -199,6 +199,13 @@ public interface WAL {
   String toString();
 
   /**
+   * In some WAL implementation, we will write WAL entries to new file if sync failed, which means,
+   * the fail recovery is depended on log roller. So here we tell the WAL that log roller has
+   * already been exited so the WAL cloud give up recovery.
+   */
+  void logRollerExited();
+
+  /**
    * When outside clients need to consume persisted WALs, they rely on a provided
    * Reader.
    */
@@ -268,7 +275,5 @@ public interface WAL {
     public String toString() {
       return this.key + "=" + this.edit;
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 08f42aa..a2761df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -76,7 +76,8 @@ public class WALFactory {
   static enum Providers {
     defaultProvider(DefaultWALProvider.class),
     filesystem(DefaultWALProvider.class),
-    multiwal(RegionGroupingProvider.class);
+    multiwal(RegionGroupingProvider.class),
+    asyncfs(AsyncFSWALProvider.class);
 
     Class<? extends WALProvider> clazz;
     Providers(Class<? extends WALProvider> clazz) {
@@ -350,9 +351,10 @@ public class WALFactory {
 
   /**
    * Create a writer for the WAL.
+   * <p>
    * should be package-private. public only for tests and
    * {@link org.apache.hadoop.hbase.regionserver.wal.Compressor}
-   * @return A WAL writer.  Close when done with it.
+   * @return A WAL writer. Close when done with it.
    * @throws IOException
    */
   public Writer createWALWriter(final FileSystem fs, final Path path) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
index 2c500dc..ad79485 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.wal;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.nio.channels.CompletionHandler;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 // imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 
@@ -80,6 +80,12 @@ public interface WALProvider {
     long getLength() throws IOException;
   }
 
+  interface AsyncWriter extends Closeable {
+    <A> void sync(CompletionHandler<Long, A> handler, A attachment);
+    void append(WAL.Entry entry);
+    long getLength();
+  }
+
   /**
    * Get number of the log files this provider is managing
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
new file mode 100644
index 0000000..7abdef9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
@@ -0,0 +1,332 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test log deletion as logs are rolled.
+ */
+public abstract class AbstractTestLogRolling  {
+  private static final Log LOG = LogFactory.getLog(AbstractTestLogRolling.class);
+  protected HRegionServer server;
+  protected String tableName;
+  protected byte[] value;
+  protected FileSystem fs;
+  protected MiniDFSCluster dfsCluster;
+  protected Admin admin;
+  protected MiniHBaseCluster cluster;
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  @Rule public final TestName name = new TestName();
+
+  public AbstractTestLogRolling()  {
+    this.server = null;
+    this.tableName = null;
+
+    String className = this.getClass().getName();
+    StringBuilder v = new StringBuilder(className);
+    while (v.length() < 1000) {
+      v.append(className);
+    }
+    this.value = Bytes.toBytes(v.toString());
+  }
+
+  // Need to override this setup so we can edit the config before it gets sent
+  // to the HDFS & HBase cluster startup.
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+
+
+    /**** configuration for testLogRolling ****/
+    // Force a region split after every 768KB
+    TEST_UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 768L * 1024L);
+
+    // We roll the log after every 32 writes
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32);
+
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.errors.tolerated", 2);
+    TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000);
+
+    // For less frequently updated regions flush after every 2 flushes
+    TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
+
+    // We flush the cache after every 8192 bytes
+    TEST_UTIL.getConfiguration().setInt(
+        HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 8192);
+
+    // Increase the amount of time between client retries
+    TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
+
+    // Reduce thread wake frequency so that other threads can get
+    // a chance to run.
+    TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    TEST_UTIL.startMiniCluster(1, 1, 2);
+
+    cluster = TEST_UTIL.getHBaseCluster();
+    dfsCluster = TEST_UTIL.getDFSCluster();
+    fs = TEST_UTIL.getTestFileSystem();
+    admin = TEST_UTIL.getHBaseAdmin();
+
+    // disable region rebalancing (interferes with log watching)
+    cluster.getMaster().balanceSwitch(false);
+  }
+
+  @After
+  public void tearDown() throws Exception  {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  protected void startAndWriteData() throws IOException, InterruptedException {
+    // When the hbase:meta table can be opened, the region servers are running
+    TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
+    this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
+
+    Table table = createTestTable(this.tableName);
+
+    server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
+    for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
+      doPut(table, i);
+      if (i % 32 == 0) {
+        // After every 32 writes sleep to let the log roller run
+        try {
+          Thread.sleep(2000);
+        } catch (InterruptedException e) {
+          // continue
+        }
+      }
+    }
+  }
+
+  /**
+   * Tests that log rolling doesn't hang when no data is written.
+   */
+  @Test(timeout=120000)
+  public void testLogRollOnNothingWritten() throws Exception {
+    final Configuration conf = TEST_UTIL.getConfiguration();
+    final WALFactory wals = new WALFactory(conf, null,
+        ServerName.valueOf("test.com",8080, 1).toString());
+    final WAL newLog = wals.getWAL(new byte[]{}, null);
+    try {
+      // Now roll the log before we write anything.
+      newLog.rollWriter(true);
+    } finally {
+      wals.close();
+    }
+  }
+
+  /**
+   * Tests that logs are deleted
+   * @throws IOException
+   * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
+   */
+  @Test
+  public void testLogRolling() throws Exception {
+    this.tableName = getName();
+    // TODO: Why does this write data take for ever?
+    startAndWriteData();
+    HRegionInfo region =
+        server.getOnlineRegions(TableName.valueOf(tableName)).get(0).getRegionInfo();
+    final WAL log = server.getWAL(region);
+    LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
+        " log files");
+
+      // flush all regions
+      for (Region r: server.getOnlineRegionsLocalContext()) {
+        r.flush(true);
+      }
+
+      // Now roll the log
+      log.rollWriter();
+
+    int count = DefaultWALProvider.getNumRolledLogFiles(log);
+    LOG.info("after flushing all regions and rolling logs there are " + count + " log files");
+      assertTrue(("actual count: " + count), count <= 2);
+  }
+
+  protected String getName() {
+    return "TestLogRolling-" + name.getMethodName();
+  }
+
+  void writeData(Table table, int rownum) throws IOException {
+    doPut(table, rownum);
+
+    // sleep to let the log roller run (if it needs to)
+    try {
+      Thread.sleep(2000);
+    } catch (InterruptedException e) {
+      // continue
+    }
+  }
+
+  void validateData(Table table, int rownum) throws IOException {
+    String row = "row" + String.format("%1$04d", rownum);
+    Get get = new Get(Bytes.toBytes(row));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result result = table.get(get);
+    assertTrue(result.size() == 1);
+    assertTrue(Bytes.equals(value,
+                result.getValue(HConstants.CATALOG_FAMILY, null)));
+    LOG.info("Validated row " + row);
+  }
+
+  void batchWriteAndWait(Table table, final FSHLog log, int start, boolean expect, int timeout)
+      throws IOException {
+    for (int i = 0; i < 10; i++) {
+      Put put = new Put(Bytes.toBytes("row"
+          + String.format("%1$04d", (start + i))));
+      put.addColumn(HConstants.CATALOG_FAMILY, null, value);
+      table.put(put);
+    }
+    Put tmpPut = new Put(Bytes.toBytes("tmprow"));
+    tmpPut.addColumn(HConstants.CATALOG_FAMILY, null, value);
+    long startTime = System.currentTimeMillis();
+    long remaining = timeout;
+    while (remaining > 0) {
+      if (log.isLowReplicationRollEnabled() == expect) {
+        break;
+      } else {
+        // Trigger calling FSHlog#checkLowReplication()
+        table.put(tmpPut);
+        try {
+          Thread.sleep(200);
+        } catch (InterruptedException e) {
+          // continue
+        }
+        remaining = timeout - (System.currentTimeMillis() - startTime);
+      }
+    }
+  }
+
+  /**
+   * Tests that logs are deleted when some region has a compaction
+   * record in WAL and no other records. See HBASE-8597.
+   */
+  @Test
+  public void testCompactionRecordDoesntBlockRolling() throws Exception {
+    Table table = null;
+
+    // When the hbase:meta table can be opened, the region servers are running
+    Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
+    try {
+      table = createTestTable(getName());
+
+      server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
+      Region region = server.getOnlineRegions(table.getName()).get(0);
+      final WAL log = server.getWAL(region.getRegionInfo());
+      Store s = region.getStore(HConstants.CATALOG_FAMILY);
+
+      //have to flush namespace to ensure it doesn't affect wall tests
+      admin.flush(TableName.NAMESPACE_TABLE_NAME);
+
+      // Put some stuff into table, to make sure we have some files to compact.
+      for (int i = 1; i <= 2; ++i) {
+        doPut(table, i);
+        admin.flush(table.getName());
+      }
+      doPut(table, 3); // don't flush yet, or compaction might trigger before we roll WAL
+      assertEquals("Should have no WAL after initial writes", 0,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+      assertEquals(2, s.getStorefilesCount());
+
+      // Roll the log and compact table, to have compaction record in the 2nd WAL.
+      log.rollWriter();
+      assertEquals("Should have WAL; one table is not flushed", 1,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+      admin.flush(table.getName());
+      region.compact(false);
+      // Wait for compaction in case if flush triggered it before us.
+      Assert.assertNotNull(s);
+      for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) {
+        Threads.sleepWithoutInterrupt(200);
+      }
+      assertEquals("Compaction didn't happen", 1, s.getStorefilesCount());
+
+      // Write some value to the table so the WAL cannot be deleted until table is flushed.
+      doPut(table, 0); // Now 2nd WAL will have both compaction and put record for table.
+      log.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
+      assertEquals("Should have WAL; one table is not flushed", 1,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+
+      // Flush table to make latest WAL obsolete; write another record, and roll again.
+      admin.flush(table.getName());
+      doPut(table, 1);
+      log.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
+      assertEquals("Should have 1 WALs at the end", 1,
+          DefaultWALProvider.getNumRolledLogFiles(log));
+    } finally {
+      if (t != null) t.close();
+      if (table != null) table.close();
+    }
+  }
+
+  protected void doPut(Table table, int i) throws IOException {
+    Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
+    put.addColumn(HConstants.CATALOG_FAMILY, null, value);
+    table.put(put);
+  }
+
+  protected Table createTestTable(String tableName) throws IOException {
+    // Create the test table and open it
+    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+    desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+    admin.createTable(desc);
+    return TEST_UTIL.getConnection().getTable(desc.getTableName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
new file mode 100644
index 0000000..a4267a0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
@@ -0,0 +1,209 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * WAL tests that can be reused across providers.
+ */
+public abstract class AbstractTestProtobufLog<W extends Closeable> {
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  protected FileSystem fs;
+  protected Path dir;
+  protected WALFactory wals;
+
+  @Rule
+  public final TestName currentTest = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    fs = TEST_UTIL.getDFSCluster().getFileSystem();
+    dir = new Path(TEST_UTIL.createRootDir(), currentTest.getMethodName());
+    wals = new WALFactory(TEST_UTIL.getConfiguration(), null, currentTest.getMethodName());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    wals.close();
+    FileStatus[] entries = fs.listStatus(new Path("/"));
+    for (FileStatus dir : entries) {
+      fs.delete(dir.getPath(), true);
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // Make block sizes small.
+    TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
+    // needed for testAppendClose()
+    TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
+    TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
+    // quicker heartbeat interval for faster DN death notification
+    TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
+    TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
+    TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
+
+    // faster failover with cluster.shutdown();fs.close() idiom
+    TEST_UTIL.getConfiguration()
+        .setInt("hbase.ipc.client.connect.max.retries", 1);
+    TEST_UTIL.getConfiguration().setInt(
+        "dfs.client.block.recovery.retries", 1);
+    TEST_UTIL.getConfiguration().setInt(
+      "hbase.ipc.client.connection.maxidletime", 500);
+    TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
+        SampleRegionWALObserver.class.getName());
+    TEST_UTIL.startMiniDFSCluster(3);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Reads the WAL with and without WALTrailer.
+   * @throws IOException
+   */
+  @Test
+  public void testWALTrailer() throws IOException {
+    // read With trailer.
+    doRead(true);
+    // read without trailer
+    doRead(false);
+  }
+
+  /**
+   * Appends entries in the WAL and reads it.
+   * @param withTrailer If 'withTrailer' is true, it calls a close on the WALwriter before reading
+   *          so that a trailer is appended to the WAL. Otherwise, it starts reading after the sync
+   *          call. This means that reader is not aware of the trailer. In this scenario, if the
+   *          reader tries to read the trailer in its next() call, it returns false from
+   *          ProtoBufLogReader.
+   * @throws IOException
+   */
+  private void doRead(boolean withTrailer) throws IOException {
+    final int columnCount = 5;
+    final int recordCount = 5;
+    final TableName tableName =
+        TableName.valueOf("tablename");
+    final byte[] row = Bytes.toBytes("row");
+    long timestamp = System.currentTimeMillis();
+    Path path = new Path(dir, "tempwal");
+    // delete the log if already exists, for test only
+    fs.delete(path, true);
+    W writer = null;
+    ProtobufLogReader reader = null;
+    try {
+      HRegionInfo hri = new HRegionInfo(tableName,
+          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+      HTableDescriptor htd = new HTableDescriptor(tableName);
+      fs.mkdirs(dir);
+      // Write log in pb format.
+      writer = createWriter(path);
+      for (int i = 0; i < recordCount; ++i) {
+        WALKey key = new WALKey(
+            hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
+        WALEdit edit = new WALEdit();
+        for (int j = 0; j < columnCount; ++j) {
+          if (i == 0) {
+            htd.addFamily(new HColumnDescriptor("column" + j));
+          }
+          String value = i + "" + j;
+          edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
+        }
+        append(writer, new WAL.Entry(key, edit));
+      }
+      sync(writer);
+      if (withTrailer) writer.close();
+
+      // Now read the log using standard means.
+      reader = (ProtobufLogReader) wals.createReader(fs, path);
+      if (withTrailer) {
+        assertNotNull(reader.trailer);
+      } else {
+        assertNull(reader.trailer);
+      }
+      for (int i = 0; i < recordCount; ++i) {
+        WAL.Entry entry = reader.next();
+        assertNotNull(entry);
+        assertEquals(columnCount, entry.getEdit().size());
+        assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
+        assertEquals(tableName, entry.getKey().getTablename());
+        int idx = 0;
+        for (Cell val : entry.getEdit().getCells()) {
+          assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
+            val.getRowLength()));
+          String value = i + "" + idx;
+          assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
+          idx++;
+        }
+      }
+      WAL.Entry entry = reader.next();
+      assertNull(entry);
+    } finally {
+      if (writer != null) {
+        writer.close();
+      }
+      if (reader != null) {
+        reader.close();
+      }
+    }
+  }
+
+  protected abstract W createWriter(Path path) throws IOException;
+
+  protected abstract void append(W writer, WAL.Entry entry) throws IOException;
+
+  protected abstract void sync(W writer) throws IOException;
+}


[4/4] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Posted by zh...@apache.org.
HBASE-15265 Implement an asynchronous FSHLog


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c96b642f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c96b642f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c96b642f

Branch: refs/heads/master
Commit: c96b642f15ddc24ad1f52616a4fb74feb51483c4
Parents: 1a9837a
Author: zhangduo <zh...@apache.org>
Authored: Sat Mar 26 15:40:45 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sat Mar 26 15:41:27 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/LogRoller.java    |    3 +
 .../hbase/regionserver/wal/AbstractFSWAL.java   |  910 +++++++++++
 .../wal/AbstractProtobufLogWriter.java          |  170 ++
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  732 +++++++++
 .../wal/AsyncProtobufLogWriter.java             |  203 +++
 .../hadoop/hbase/regionserver/wal/FSHLog.java   | 1518 ++++--------------
 .../hbase/regionserver/wal/FSWALEntry.java      |   24 +-
 .../regionserver/wal/ProtobufLogWriter.java     |  148 +-
 .../hbase/regionserver/wal/SyncFuture.java      |  137 +-
 .../hbase/regionserver/wal/WriterBase.java      |   62 -
 .../FanOutOneBlockAsyncDFSOutputHelper.java     |   40 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |  361 +++++
 .../hadoop/hbase/wal/AsyncFSWALProvider.java    |   73 +
 .../hadoop/hbase/wal/DefaultWALProvider.java    |  348 +---
 .../hadoop/hbase/wal/DisabledWALProvider.java   |    4 +
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |    9 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java |    6 +-
 .../apache/hadoop/hbase/wal/WALProvider.java    |   10 +-
 .../wal/AbstractTestLogRolling.java             |  332 ++++
 .../wal/AbstractTestProtobufLog.java            |  209 +++
 .../regionserver/wal/SequenceFileLogWriter.java |   22 +-
 .../wal/TestAsyncLogRollPeriod.java             |   36 +
 .../regionserver/wal/TestAsyncLogRolling.java   |   65 +
 .../regionserver/wal/TestAsyncProtobufLog.java  |   82 +
 .../regionserver/wal/TestAsyncWALReplay.java    |   36 +
 .../wal/TestAsyncWALReplayCompressed.java       |   38 +
 .../hbase/regionserver/wal/TestDurability.java  |   29 +-
 .../regionserver/wal/TestLogRollAbort.java      |    6 +-
 .../regionserver/wal/TestLogRollPeriod.java     |   12 +-
 .../hbase/regionserver/wal/TestLogRolling.java  |  404 +----
 .../hbase/regionserver/wal/TestProtobufLog.java |  187 +--
 .../wal/TestWALReplayCompressed.java            |    1 -
 32 files changed, 3918 insertions(+), 2299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
index 40edc05..fa217ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
@@ -167,6 +167,9 @@ public class LogRoller extends HasThread {
         }
       }
     }
+    for (WAL wal : walNeedsRoll.keySet()) {
+      wal.logRollerExited();
+    }
     LOG.info("LogRoller exiting.");
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
new file mode 100644
index 0000000..f189ff1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -0,0 +1,910 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryUsage;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.DrainBarrier;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.htrace.NullScope;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
+ * WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
+ * This is done internal to the implementation.
+ * <p>
+ * As data is flushed from the MemStore to other on-disk structures (files sorted by key, hfiles), a
+ * WAL becomes obsolete. We can let go of all the log edits/entries for a given HRegion-sequence id.
+ * A bunch of work in the below is done keeping account of these region sequence ids -- what is
+ * flushed out to hfiles, and what is yet in WAL and in memory only.
+ * <p>
+ * It is only practical to delete entire files. Thus, we delete an entire on-disk file
+ * <code>F</code> when all of the edits in <code>F</code> have a log-sequence-id that's older
+ * (smaller) than the most-recent flush.
+ * <p>
+ * To read an WAL, call
+ * {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path)}. *
+ * <h2>Failure Semantic</h2> If an exception on append or sync, roll the WAL because the current WAL
+ * is now a lame duck; any more appends or syncs will fail also with the same original exception. If
+ * we have made successful appends to the WAL and we then are unable to sync them, our current
+ * semantic is to return error to the client that the appends failed but also to abort the current
+ * context, usually the hosting server. We need to replay the WALs. <br>
+ * TODO: Change this semantic. A roll of WAL may be sufficient as long as we have flagged client
+ * that the append failed. <br>
+ * TODO: replication may pick up these last edits though they have been marked as failed append
+ * (Need to keep our own file lengths, not rely on HDFS).
+ */
+@InterfaceAudience.Private
+public abstract class AbstractFSWAL<W> implements WAL {
+
+  private static final Log LOG = LogFactory.getLog(AbstractFSWAL.class);
+
+  protected static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
+
+  /**
+   * file system instance
+   */
+  protected final FileSystem fs;
+
+  /**
+   * WAL directory, where all WAL files would be placed.
+   */
+  protected final Path walDir;
+
+  /**
+   * dir path where old logs are kept.
+   */
+  protected final Path walArchiveDir;
+
+  /**
+   * Matches just those wal files that belong to this wal instance.
+   */
+  protected final PathFilter ourFiles;
+
+  /**
+   * Prefix of a WAL file, usually the region server name it is hosted on.
+   */
+  protected final String walFilePrefix;
+
+  /**
+   * Suffix included on generated wal file names
+   */
+  protected final String walFileSuffix;
+
+  /**
+   * Prefix used when checking for wal membership.
+   */
+  protected final String prefixPathStr;
+
+  protected final WALCoprocessorHost coprocessorHost;
+
+  /**
+   * conf object
+   */
+  protected final Configuration conf;
+
+  /** Listeners that are called on WAL events. */
+  protected final List<WALActionsListener> listeners =
+      new CopyOnWriteArrayList<WALActionsListener>();
+
+  /**
+   * Class that does accounting of sequenceids in WAL subsystem. Holds oldest outstanding sequence
+   * id as yet not flushed as well as the most recent edit sequence id appended to the WAL. Has
+   * facility for answering questions such as "Is it safe to GC a WAL?".
+   */
+  protected final SequenceIdAccounting sequenceIdAccounting = new SequenceIdAccounting();
+
+  /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */
+  protected final DrainBarrier closeBarrier = new DrainBarrier();
+
+  protected final int slowSyncNs;
+
+  // If > than this size, roll the log.
+  protected final long logrollsize;
+
+  /*
+   * If more than this many logs, force flush of oldest region to oldest edit goes to disk. If too
+   * many and we crash, then will take forever replaying. Keep the number of logs tidy.
+   */
+  protected final int maxLogs;
+
+  /**
+   * This lock makes sure only one log roll runs at a time. Should not be taken while any other lock
+   * is held. We don't just use synchronized because that results in bogus and tedious findbugs
+   * warning when it thinks synchronized controls writer thread safety. It is held when we are
+   * actually rolling the log. It is checked when we are looking to see if we should roll the log or
+   * not.
+   */
+  protected final ReentrantLock rollWriterLock = new ReentrantLock(true);
+
+  // The timestamp (in ms) when the log file was created.
+  protected final AtomicLong filenum = new AtomicLong(-1);
+
+  // Number of transactions in the current Wal.
+  protected final AtomicInteger numEntries = new AtomicInteger(0);
+
+  /**
+   * The highest known outstanding unsync'd WALEdit transaction id. Usually, we use a queue to pass
+   * WALEdit to background consumer thread, and the transaction id is the sequence number of the
+   * corresponding entry in queue.
+   */
+  protected volatile long highestUnsyncedTxid = -1;
+
+  /**
+   * Updated to the transaction id of the last successful sync call. This can be less than
+   * {@link #highestUnsyncedTxid} for case where we have an append where a sync has not yet come in
+   * for it.
+   */
+  protected final AtomicLong highestSyncedTxid = new AtomicLong(0);
+
+  /**
+   * The total size of wal
+   */
+  protected final AtomicLong totalLogSize = new AtomicLong(0);
+  /**
+   * Current log file.
+   */
+  volatile W writer;
+
+  protected volatile boolean closed = false;
+
+  protected final AtomicBoolean shutdown = new AtomicBoolean(false);
+  /**
+   * WAL Comparator; it compares the timestamp (log filenum), present in the log file name. Throws
+   * an IllegalArgumentException if used to compare paths from different wals.
+   */
+  final Comparator<Path> LOG_NAME_COMPARATOR = new Comparator<Path>() {
+    @Override
+    public int compare(Path o1, Path o2) {
+      return Long.compare(getFileNumFromFileName(o1), getFileNumFromFileName(o2));
+    }
+  };
+
+  /**
+   * Map of WAL log file to the latest sequence ids of all regions it has entries of. The map is
+   * sorted by the log file creation timestamp (contained in the log file name).
+   */
+  protected ConcurrentNavigableMap<Path, Map<byte[], Long>> byWalRegionSequenceIds =
+      new ConcurrentSkipListMap<Path, Map<byte[], Long>>(LOG_NAME_COMPARATOR);
+
+  /**
+   * Map of {@link SyncFuture}s keyed by Handler objects. Used so we reuse SyncFutures.
+   * <p>
+   * TODO: Reus FSWALEntry's rather than create them anew each time as we do SyncFutures here.
+   * <p>
+   * TODO: Add a FSWalEntry and SyncFuture as thread locals on handlers rather than have them get
+   * them from this Map?
+   */
+  private final ConcurrentMap<Thread, SyncFuture> syncFuturesByHandler;
+
+  /**
+   * A log file has a creation timestamp (in ms) in its file name ({@link #filenum}. This helper
+   * method returns the creation timestamp from a given log file. It extracts the timestamp assuming
+   * the filename is created with the {@link #computeFilename(long filenum)} method.
+   * @return timestamp, as in the log file name.
+   */
+  protected long getFileNumFromFileName(Path fileName) {
+    if (fileName == null) {
+      throw new IllegalArgumentException("file name can't be null");
+    }
+    if (!ourFiles.accept(fileName)) {
+      throw new IllegalArgumentException("The log file " + fileName
+          + " doesn't belong to this WAL. (" + toString() + ")");
+    }
+    final String fileNameString = fileName.toString();
+    String chompedPath =
+        fileNameString.substring(prefixPathStr.length(),
+          (fileNameString.length() - walFileSuffix.length()));
+    return Long.parseLong(chompedPath);
+  }
+
+  private int calculateMaxLogFiles(float memstoreSizeRatio, long logRollSize) {
+    MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
+    return Math.round(mu.getMax() * memstoreSizeRatio * 2 / logRollSize);
+  }
+
+  protected AbstractFSWAL(final FileSystem fs, final Path rootDir, final String logDir,
+      final String archiveDir, final Configuration conf, final List<WALActionsListener> listeners,
+      final boolean failIfWALExists, final String prefix, final String suffix)
+      throws FailedLogCloseException, IOException {
+    this.fs = fs;
+    this.walDir = new Path(rootDir, logDir);
+    this.walArchiveDir = new Path(rootDir, archiveDir);
+    this.conf = conf;
+
+    if (!fs.exists(walDir) && !fs.mkdirs(walDir)) {
+      throw new IOException("Unable to mkdir " + walDir);
+    }
+
+    if (!fs.exists(this.walArchiveDir)) {
+      if (!fs.mkdirs(this.walArchiveDir)) {
+        throw new IOException("Unable to mkdir " + this.walArchiveDir);
+      }
+    }
+
+    // If prefix is null||empty then just name it wal
+    this.walFilePrefix =
+        prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8");
+    // we only correctly differentiate suffices when numeric ones start with '.'
+    if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) {
+      throw new IllegalArgumentException("WAL suffix must start with '" + WAL_FILE_NAME_DELIMITER
+          + "' but instead was '" + suffix + "'");
+    }
+    // Now that it exists, set the storage policy for the entire directory of wal files related to
+    // this FSHLog instance
+    FSUtils.setStoragePolicy(fs, conf, this.walDir, HConstants.WAL_STORAGE_POLICY,
+      HConstants.DEFAULT_WAL_STORAGE_POLICY);
+    this.walFileSuffix = (suffix == null) ? "" : URLEncoder.encode(suffix, "UTF8");
+    this.prefixPathStr = new Path(walDir, walFilePrefix + WAL_FILE_NAME_DELIMITER).toString();
+
+    this.ourFiles = new PathFilter() {
+      @Override
+      public boolean accept(final Path fileName) {
+        // The path should start with dir/<prefix> and end with our suffix
+        final String fileNameString = fileName.toString();
+        if (!fileNameString.startsWith(prefixPathStr)) {
+          return false;
+        }
+        if (walFileSuffix.isEmpty()) {
+          // in the case of the null suffix, we need to ensure the filename ends with a timestamp.
+          return org.apache.commons.lang.StringUtils.isNumeric(fileNameString
+              .substring(prefixPathStr.length()));
+        } else if (!fileNameString.endsWith(walFileSuffix)) {
+          return false;
+        }
+        return true;
+      }
+    };
+
+    if (failIfWALExists) {
+      final FileStatus[] walFiles = FSUtils.listStatus(fs, walDir, ourFiles);
+      if (null != walFiles && 0 != walFiles.length) {
+        throw new IOException("Target WAL already exists within directory " + walDir);
+      }
+    }
+
+    // Register listeners. TODO: Should this exist anymore? We have CPs?
+    if (listeners != null) {
+      for (WALActionsListener i : listeners) {
+        registerWALActionsListener(i);
+      }
+    }
+    this.coprocessorHost = new WALCoprocessorHost(this, conf);
+
+    // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks
+    // (it costs a little x'ing bocks)
+    final long blocksize =
+        this.conf.getLong("hbase.regionserver.hlog.blocksize",
+          FSUtils.getDefaultBlockSize(this.fs, this.walDir));
+    this.logrollsize =
+        (long) (blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
+
+    float memstoreRatio =
+        conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY, conf.getFloat(
+          HeapMemorySizeUtil.MEMSTORE_SIZE_OLD_KEY, HeapMemorySizeUtil.DEFAULT_MEMSTORE_SIZE));
+    boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null;
+    if (maxLogsDefined) {
+      LOG.warn("'hbase.regionserver.maxlogs' was deprecated.");
+    }
+    this.maxLogs =
+        conf.getInt("hbase.regionserver.maxlogs",
+          Math.max(32, calculateMaxLogFiles(memstoreRatio, logrollsize)));
+
+    LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) + ", rollsize="
+        + StringUtils.byteDesc(this.logrollsize) + ", prefix=" + this.walFilePrefix + ", suffix="
+        + walFileSuffix + ", logDir=" + this.walDir + ", archiveDir=" + this.walArchiveDir);
+    this.slowSyncNs =
+        1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms", DEFAULT_SLOW_SYNC_TIME_MS);
+    int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
+    // Presize our map of SyncFutures by handler objects.
+    this.syncFuturesByHandler = new ConcurrentHashMap<Thread, SyncFuture>(maxHandlersCount);
+  }
+
+  @Override
+  public void registerWALActionsListener(WALActionsListener listener) {
+    this.listeners.add(listener);
+  }
+
+  @Override
+  public boolean unregisterWALActionsListener(WALActionsListener listener) {
+    return this.listeners.remove(listener);
+  }
+
+  @Override
+  public WALCoprocessorHost getCoprocessorHost() {
+    return coprocessorHost;
+  }
+
+  @Override
+  public Long startCacheFlush(byte[] encodedRegionName, Set<byte[]> families) {
+    if (!closeBarrier.beginOp()) {
+      LOG.info("Flush not started for " + Bytes.toString(encodedRegionName) + "; server closing.");
+      return null;
+    }
+    return this.sequenceIdAccounting.startCacheFlush(encodedRegionName, families);
+  }
+
+  @Override
+  public void completeCacheFlush(byte[] encodedRegionName) {
+    this.sequenceIdAccounting.completeCacheFlush(encodedRegionName);
+    closeBarrier.endOp();
+  }
+
+  @Override
+  public void abortCacheFlush(byte[] encodedRegionName) {
+    this.sequenceIdAccounting.abortCacheFlush(encodedRegionName);
+    closeBarrier.endOp();
+  }
+
+  @Override
+  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
+    // Used by tests. Deprecated as too subtle for general usage.
+    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName);
+  }
+
+  @Override
+  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) {
+    // This method is used by tests and for figuring if we should flush or not because our
+    // sequenceids are too old. It is also used reporting the master our oldest sequenceid for use
+    // figuring what edits can be skipped during log recovery. getEarliestMemStoreSequenceId
+    // from this.sequenceIdAccounting is looking first in flushingOldestStoreSequenceIds, the
+    // currently flushing sequence ids, and if anything found there, it is returning these. This is
+    // the right thing to do for the reporting oldest sequenceids to master; we won't skip edits if
+    // we crash during the flush. For figuring what to flush, we might get requeued if our sequence
+    // id is old even though we are currently flushing. This may mean we do too much flushing.
+    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName, familyName);
+  }
+
+  @Override
+  public byte[][] rollWriter() throws FailedLogCloseException, IOException {
+    return rollWriter(false);
+  }
+
+  /**
+   * This is a convenience method that computes a new filename with a given file-number.
+   * @param filenum to use
+   * @return Path
+   */
+  protected Path computeFilename(final long filenum) {
+    if (filenum < 0) {
+      throw new RuntimeException("WAL file number can't be < 0");
+    }
+    String child = walFilePrefix + WAL_FILE_NAME_DELIMITER + filenum + walFileSuffix;
+    return new Path(walDir, child);
+  }
+
+  /**
+   * This is a convenience method that computes a new filename with a given using the current WAL
+   * file-number
+   * @return Path
+   */
+  public Path getCurrentFileName() {
+    return computeFilename(this.filenum.get());
+  }
+
+  /**
+   * retrieve the next path to use for writing. Increments the internal filenum.
+   */
+  private Path getNewPath() throws IOException {
+    this.filenum.set(System.currentTimeMillis());
+    Path newPath = getCurrentFileName();
+    while (fs.exists(newPath)) {
+      this.filenum.incrementAndGet();
+      newPath = getCurrentFileName();
+    }
+    return newPath;
+  }
+
+  @VisibleForTesting
+  Path getOldPath() {
+    long currentFilenum = this.filenum.get();
+    Path oldPath = null;
+    if (currentFilenum > 0) {
+      // ComputeFilename will take care of meta wal filename
+      oldPath = computeFilename(currentFilenum);
+    } // I presume if currentFilenum is <= 0, this is first file and null for oldPath if fine?
+    return oldPath;
+  }
+
+  /**
+   * Tell listeners about pre log roll.
+   */
+  private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
+      throws IOException {
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.preLogRoll(oldPath, newPath);
+      }
+    }
+  }
+
+  /**
+   * Tell listeners about post log roll.
+   */
+  private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
+      throws IOException {
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.postLogRoll(oldPath, newPath);
+      }
+    }
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  /** @return the number of rolled log files */
+  public int getNumRolledLogFiles() {
+    return byWalRegionSequenceIds.size();
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  /** @return the number of log files in use */
+  public int getNumLogFiles() {
+    // +1 for current use log
+    return getNumRolledLogFiles() + 1;
+  }
+
+  /**
+   * If the number of un-archived WAL files is greater than maximum allowed, check the first
+   * (oldest) WAL file, and returns those regions which should be flushed so that it can be
+   * archived.
+   * @return regions (encodedRegionNames) to flush in order to archive oldest WAL file.
+   */
+  byte[][] findRegionsToForceFlush() throws IOException {
+    byte[][] regions = null;
+    int logCount = getNumRolledLogFiles();
+    if (logCount > this.maxLogs && logCount > 0) {
+      Map.Entry<Path, Map<byte[], Long>> firstWALEntry = this.byWalRegionSequenceIds.firstEntry();
+      regions = this.sequenceIdAccounting.findLower(firstWALEntry.getValue());
+    }
+    if (regions != null) {
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < regions.length; i++) {
+        if (i > 0) {
+          sb.append(", ");
+        }
+        sb.append(Bytes.toStringBinary(regions[i]));
+      }
+      LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs + "; forcing flush of "
+          + regions.length + " regions(s): " + sb.toString());
+    }
+    return regions;
+  }
+
+  /**
+   * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
+   */
+  private void cleanOldLogs() throws IOException {
+    List<Path> logsToArchive = null;
+    // For each log file, look at its Map of regions to highest sequence id; if all sequence ids
+    // are older than what is currently in memory, the WAL can be GC'd.
+    for (Map.Entry<Path, Map<byte[], Long>> e : this.byWalRegionSequenceIds.entrySet()) {
+      Path log = e.getKey();
+      Map<byte[], Long> sequenceNums = e.getValue();
+      if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
+        if (logsToArchive == null) {
+          logsToArchive = new ArrayList<Path>();
+        }
+        logsToArchive.add(log);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("WAL file ready for archiving " + log);
+        }
+      }
+    }
+    if (logsToArchive != null) {
+      for (Path p : logsToArchive) {
+        this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
+        archiveLogFile(p);
+        this.byWalRegionSequenceIds.remove(p);
+      }
+    }
+  }
+
+  /*
+   * only public so WALSplitter can use.
+   * @return archived location of a WAL file with the given path p
+   */
+  public static Path getWALArchivePath(Path archiveDir, Path p) {
+    return new Path(archiveDir, p.getName());
+  }
+
+  private void archiveLogFile(final Path p) throws IOException {
+    Path newPath = getWALArchivePath(this.walArchiveDir, p);
+    // Tell our listeners that a log is going to be archived.
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.preLogArchive(p, newPath);
+      }
+    }
+    LOG.info("Archiving " + p + " to " + newPath);
+    if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
+      throw new IOException("Unable to rename " + p + " to " + newPath);
+    }
+    // Tell our listeners that a log has been archived.
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.postLogArchive(p, newPath);
+      }
+    }
+  }
+
+  /**
+   * Cleans up current writer closing it and then puts in place the passed in
+   * <code>nextWriter</code>.
+   * <p>
+   * <ul>
+   * <li>In the case of creating a new WAL, oldPath will be null.</li>
+   * <li>In the case of rolling over from one file to the next, none of the parameters will be null.
+   * </li>
+   * <li>In the case of closing out this FSHLog with no further use newPath and nextWriter will be
+   * null.</li>
+   * </ul>
+   * @param oldPath may be null
+   * @param newPath may be null
+   * @param nextWriter may be null
+   * @return the passed in <code>newPath</code>
+   * @throws IOException if there is a problem flushing or closing the underlying FS
+   */
+  Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
+    TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
+    try {
+      long oldFileLen = 0L;
+      doReplaceWriter(oldPath, newPath, nextWriter);
+      int oldNumEntries = this.numEntries.get();
+      final String newPathString = (null == newPath ? null : FSUtils.getPath(newPath));
+      if (oldPath != null) {
+        this.byWalRegionSequenceIds.put(oldPath, this.sequenceIdAccounting.resetHighest());
+        this.totalLogSize.addAndGet(oldFileLen);
+        LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries
+            + ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " + newPathString);
+      } else {
+        LOG.info("New WAL " + newPathString);
+      }
+      return newPath;
+    } finally {
+      scope.close();
+    }
+  }
+
+  protected Span blockOnSync(final SyncFuture syncFuture) throws IOException {
+    // Now we have published the ringbuffer, halt the current thread until we get an answer back.
+    try {
+      syncFuture.get();
+      return syncFuture.getSpan();
+    } catch (InterruptedException ie) {
+      LOG.warn("Interrupted", ie);
+      throw convertInterruptedExceptionToIOException(ie);
+    } catch (ExecutionException e) {
+      throw ensureIOException(e.getCause());
+    }
+  }
+
+  private static IOException ensureIOException(final Throwable t) {
+    return (t instanceof IOException) ? (IOException) t : new IOException(t);
+  }
+
+  private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
+    Thread.currentThread().interrupt();
+    IOException ioe = new InterruptedIOException();
+    ioe.initCause(ie);
+    return ioe;
+  }
+
+  @Override
+  public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
+    rollWriterLock.lock();
+    try {
+      // Return if nothing to flush.
+      if (!force && this.writer != null && this.numEntries.get() <= 0) {
+        return null;
+      }
+      byte[][] regionsToFlush = null;
+      if (this.closed) {
+        LOG.debug("WAL closed. Skipping rolling of writer");
+        return regionsToFlush;
+      }
+      if (!closeBarrier.beginOp()) {
+        LOG.debug("WAL closing. Skipping rolling of writer");
+        return regionsToFlush;
+      }
+      TraceScope scope = Trace.startSpan("FSHLog.rollWriter");
+      try {
+        Path oldPath = getOldPath();
+        Path newPath = getNewPath();
+        // Any exception from here on is catastrophic, non-recoverable so we currently abort.
+        W nextWriter = this.createWriterInstance(newPath);
+        tellListenersAboutPreLogRoll(oldPath, newPath);
+        // NewPath could be equal to oldPath if replaceWriter fails.
+        newPath = replaceWriter(oldPath, newPath, nextWriter);
+        tellListenersAboutPostLogRoll(oldPath, newPath);
+        // Can we delete any of the old log files?
+        if (getNumRolledLogFiles() > 0) {
+          cleanOldLogs();
+          regionsToFlush = findRegionsToForceFlush();
+        }
+      } finally {
+        closeBarrier.endOp();
+        assert scope == NullScope.INSTANCE || !scope.isDetached();
+        scope.close();
+      }
+      return regionsToFlush;
+    } finally {
+      rollWriterLock.unlock();
+    }
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  /** @return the size of log files in use */
+  public long getLogFileSize() {
+    return this.totalLogSize.get();
+  }
+
+  // public only until class moves to o.a.h.h.wal
+  public void requestLogRoll() {
+    requestLogRoll(false);
+  }
+
+  /**
+   * Get the backing files associated with this WAL.
+   * @return may be null if there are no files.
+   */
+  protected FileStatus[] getFiles() throws IOException {
+    return FSUtils.listStatus(fs, walDir, ourFiles);
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+    if (!shutdown.compareAndSet(false, true)) {
+      return;
+    }
+    closed = true;
+    try {
+      // Prevent all further flushing and rolling.
+      closeBarrier.stopAndDrainOps();
+    } catch (InterruptedException e) {
+      LOG.error("Exception while waiting for cache flushes and log rolls", e);
+      Thread.currentThread().interrupt();
+    }
+    // Tell our listeners that the log is closing
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.logCloseRequested();
+      }
+    }
+    doShutdown();
+  }
+
+  @Override
+  public void close() throws IOException {
+    shutdown();
+    final FileStatus[] files = getFiles();
+    if (null != files && 0 != files.length) {
+      for (FileStatus file : files) {
+        Path p = getWALArchivePath(this.walArchiveDir, file.getPath());
+        // Tell our listeners that a log is going to be archived.
+        if (!this.listeners.isEmpty()) {
+          for (WALActionsListener i : this.listeners) {
+            i.preLogArchive(file.getPath(), p);
+          }
+        }
+
+        if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
+          throw new IOException("Unable to rename " + file.getPath() + " to " + p);
+        }
+        // Tell our listeners that a log was archived.
+        if (!this.listeners.isEmpty()) {
+          for (WALActionsListener i : this.listeners) {
+            i.postLogArchive(file.getPath(), p);
+          }
+        }
+      }
+      LOG.debug("Moved " + files.length + " WAL file(s) to " + FSUtils.getPath(this.walArchiveDir));
+    }
+    LOG.info("Closed WAL: " + toString());
+  }
+
+  protected SyncFuture getSyncFuture(final long sequence, Span span) {
+    SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
+    if (syncFuture == null) {
+      syncFuture = new SyncFuture();
+      this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
+    }
+    return syncFuture.reset(sequence, span);
+  }
+
+  protected void requestLogRoll(boolean tooFewReplicas) {
+    if (!this.listeners.isEmpty()) {
+      for (WALActionsListener i : this.listeners) {
+        i.logRollRequested(tooFewReplicas);
+      }
+    }
+  }
+
+  long getUnflushedEntriesCount() {
+    long highestSynced = this.highestSyncedTxid.get();
+    long highestUnsynced = this.highestUnsyncedTxid;
+    return highestSynced >= highestUnsynced ? 0 : highestUnsynced - highestSynced;
+  }
+
+  boolean isUnflushedEntries() {
+    return getUnflushedEntriesCount() > 0;
+  }
+
+  /**
+   * Exposed for testing only. Use to tricks like halt the ring buffer appending.
+   */
+  @VisibleForTesting
+  void atHeadOfRingBufferEventHandlerAppend() {
+    // Noop
+  }
+
+  protected boolean append(W writer, FSWALEntry entry) throws IOException {
+    // TODO: WORK ON MAKING THIS APPEND FASTER. DOING WAY TOO MUCH WORK WITH CPs, PBing, etc.
+    atHeadOfRingBufferEventHandlerAppend();
+    long start = EnvironmentEdgeManager.currentTime();
+    byte[] encodedRegionName = entry.getKey().getEncodedRegionName();
+    long regionSequenceId = WALKey.NO_SEQUENCE_ID;
+    // We are about to append this edit; update the region-scoped sequence number. Do it
+    // here inside this single appending/writing thread. Events are ordered on the ringbuffer
+    // so region sequenceids will also be in order.
+    regionSequenceId = entry.stampRegionSequenceId();
+    // Edits are empty, there is nothing to append. Maybe empty when we are looking for a
+    // region sequence id only, a region edit/sequence id that is not associated with an actual
+    // edit. It has to go through all the rigmarole to be sure we have the right ordering.
+    if (entry.getEdit().isEmpty()) {
+      return false;
+    }
+
+    // Coprocessor hook.
+    if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit())) {
+      if (entry.getEdit().isReplay()) {
+        // Set replication scope null so that this won't be replicated
+        entry.getKey().serializeReplicationScope(false);
+      }
+    }
+    if (!listeners.isEmpty()) {
+      for (WALActionsListener i : listeners) {
+        i.visitLogEntryBeforeWrite(entry.getKey(), entry.getEdit());
+      }
+    }
+    doAppend(writer, entry);
+    assert highestUnsyncedTxid < entry.getTxid();
+    highestUnsyncedTxid = entry.getTxid();
+    sequenceIdAccounting.update(encodedRegionName, entry.getFamilyNames(), regionSequenceId,
+      entry.isInMemstore());
+    coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit());
+    // Update metrics.
+    postAppend(entry, EnvironmentEdgeManager.currentTime() - start);
+    numEntries.incrementAndGet();
+    return true;
+  }
+
+  private long postAppend(final Entry e, final long elapsedTime) {
+    long len = 0;
+    if (!listeners.isEmpty()) {
+      for (Cell cell : e.getEdit().getCells()) {
+        len += CellUtil.estimatedSerializedSizeOf(cell);
+      }
+      for (WALActionsListener listener : listeners) {
+        listener.postAppend(len, elapsedTime);
+      }
+    }
+    return len;
+  }
+
+  protected void postSync(final long timeInNanos, final int handlerSyncs) {
+    if (timeInNanos > this.slowSyncNs) {
+      String msg =
+          new StringBuilder().append("Slow sync cost: ").append(timeInNanos / 1000000)
+              .append(" ms, current pipeline: ").append(Arrays.toString(getPipeline())).toString();
+      Trace.addTimelineAnnotation(msg);
+      LOG.info(msg);
+    }
+    if (!listeners.isEmpty()) {
+      for (WALActionsListener listener : listeners) {
+        listener.postSync(timeInNanos, handlerSyncs);
+      }
+    }
+  }
+
+  /**
+   * NOTE: This append, at a time that is usually after this call returns, starts an mvcc
+   * transaction by calling 'begin' wherein which we assign this update a sequenceid. At assignment
+   * time, we stamp all the passed in Cells inside WALEdit with their sequenceId. You must
+   * 'complete' the transaction this mvcc transaction by calling
+   * MultiVersionConcurrencyControl#complete(...) or a variant otherwise mvcc will get stuck. Do it
+   * in the finally of a try/finally block within which this append lives and any subsequent
+   * operations like sync or update of memstore, etc. Get the WriteEntry to pass mvcc out of the
+   * passed in WALKey <code>walKey</code> parameter. Be warned that the WriteEntry is not
+   * immediately available on return from this method. It WILL be available subsequent to a sync of
+   * this append; otherwise, you will just have to wait on the WriteEntry to get filled in.
+   */
+  @Override
+  public abstract long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
+      throws IOException;
+
+  protected abstract void doAppend(W writer, FSWALEntry entry) throws IOException;
+
+  protected abstract W createWriterInstance(Path path) throws IOException;
+
+  /**
+   * @return old wal file size
+   */
+  protected abstract long doReplaceWriter(Path oldPath, Path newPath, W nextWriter)
+      throws IOException;
+
+  protected abstract void doShutdown() throws IOException;
+
+  /**
+   * This method gets the pipeline for the current WAL.
+   */
+  @VisibleForTesting
+  abstract DatanodeInfo[] getPipeline();
+
+  /**
+   * This method gets the datanode replication count for the current WAL.
+   */
+  @VisibleForTesting
+  abstract int getLogReplication();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
new file mode 100644
index 0000000..66f1f54
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.DEFAULT_WAL_TRAILER_WARN_SIZE;
+import static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRAILER_WARN_SIZE;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.io.util.LRUDictionary;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * Base class for Protobuf log writer.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public abstract class AbstractProtobufLogWriter {
+
+  private static final Log LOG = LogFactory.getLog(AbstractProtobufLogWriter.class);
+
+  protected CompressionContext compressionContext;
+  protected Configuration conf;
+  protected Codec.Encoder cellEncoder;
+  protected WALCellCodec.ByteStringCompressor compressor;
+  protected boolean trailerWritten;
+  protected WALTrailer trailer;
+  // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
+  // than this size, it is written/read respectively, with a WARN message in the log.
+  protected int trailerWarnSize;
+
+  protected AtomicLong length = new AtomicLong();
+
+  private WALCellCodec getCodec(Configuration conf, CompressionContext compressionContext)
+      throws IOException {
+    return WALCellCodec.create(conf, null, compressionContext);
+  }
+
+  protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder)
+      throws IOException {
+    if (!builder.hasWriterClsName()) {
+      builder.setWriterClsName(ProtobufLogWriter.class.getSimpleName());
+    }
+    if (!builder.hasCellCodecClsName()) {
+      builder.setCellCodecClsName(WALCellCodec.getWALCellCodecClass(conf));
+    }
+    return builder.build();
+  }
+
+  private boolean initializeCompressionContext(Configuration conf, Path path) throws IOException {
+    boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
+    if (doCompress) {
+      try {
+        this.compressionContext = new CompressionContext(LRUDictionary.class,
+            FSUtils.isRecoveredEdits(path),
+            conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
+      } catch (Exception e) {
+        throw new IOException("Failed to initiate CompressionContext", e);
+      }
+    }
+    return doCompress;
+  }
+
+  public void init(FileSystem fs, Path path, Configuration conf, boolean overwritable)
+      throws IOException {
+    this.conf = conf;
+    boolean doCompress = initializeCompressionContext(conf, path);
+    this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
+    int bufferSize = FSUtils.getDefaultBufferSize(fs);
+    short replication = (short) conf.getInt("hbase.regionserver.hlog.replication",
+      FSUtils.getDefaultReplication(fs, path));
+    long blockSize = conf.getLong("hbase.regionserver.hlog.blocksize",
+      FSUtils.getDefaultBlockSize(fs, path));
+
+    initOutput(fs, path, overwritable, bufferSize, replication, blockSize);
+
+    boolean doTagCompress = doCompress
+        && conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true);
+    length.set(writeMagicAndWALHeader(ProtobufLogReader.PB_WAL_MAGIC, buildWALHeader(conf,
+      WALHeader.newBuilder().setHasCompression(doCompress).setHasTagCompression(doTagCompress))));
+
+    initAfterHeader(doCompress);
+
+    // instantiate trailer to default value.
+    trailer = WALTrailer.newBuilder().build();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Initialized protobuf WAL=" + path + ", compression=" + doCompress);
+    }
+  }
+
+  protected void initAfterHeader(boolean doCompress) throws IOException {
+    WALCellCodec codec = getCodec(conf, this.compressionContext);
+    this.cellEncoder = codec.getEncoder(getOutputStreamForCellEncoder());
+    if (doCompress) {
+      this.compressor = codec.getByteStringCompressor();
+    }
+  }
+
+  void setWALTrailer(WALTrailer walTrailer) {
+    this.trailer = walTrailer;
+  }
+
+  public long getLength() {
+    return length.get();
+  }
+
+  private WALTrailer buildWALTrailer(WALTrailer.Builder builder) {
+    return builder.build();
+  }
+
+  protected void writeWALTrailer() {
+    try {
+      int trailerSize = 0;
+      if (this.trailer == null) {
+        // use default trailer.
+        LOG.warn("WALTrailer is null. Continuing with default.");
+        this.trailer = buildWALTrailer(WALTrailer.newBuilder());
+        trailerSize = this.trailer.getSerializedSize();
+      } else if ((trailerSize = this.trailer.getSerializedSize()) > this.trailerWarnSize) {
+        // continue writing after warning the user.
+        LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum size : " + trailerSize
+            + " > " + this.trailerWarnSize);
+      }
+      length.set(writeWALTrailerAndMagic(trailer, ProtobufLogReader.PB_WAL_COMPLETE_MAGIC));
+      this.trailerWritten = true;
+    } catch (IOException ioe) {
+      LOG.warn("Failed to write trailer, non-fatal, continuing...", ioe);
+    }
+  }
+
+  protected abstract void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
+      short replication, long blockSize) throws IOException;
+
+  /**
+   * return the file length after written.
+   */
+  protected abstract long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException;
+
+  protected abstract long writeWALTrailerAndMagic(WALTrailer trailer, byte[] magic)
+      throws IOException;
+
+  protected abstract OutputStream getOutputStreamForCellEncoder();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
new file mode 100644
index 0000000..b80f2c9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -0,0 +1,732 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.apache.hadoop.hbase.HConstants.REGION_SERVER_HANDLER_COUNT;
+import static org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.shouldRetryCreate;
+import io.netty.channel.EventLoop;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.Promise;
+import io.netty.util.concurrent.ScheduledFuture;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.channels.CompletionHandler;
+import java.util.ArrayDeque;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutput;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.NameNodeException;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.htrace.NullScope;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * An asynchronous implementation of FSWAL.
+ * <p>
+ * Here 'waitingConsumePayloads' acts as the RingBuffer in FSHLog. We do not use RingBuffer here
+ * because RingBuffer need an exclusive thread to consume the entries in it, and here we want to run
+ * the append and sync operation inside EventLoop. We can not use EventLoop as the RingBuffer's
+ * executor otherwise the EventLoop can not process any other events such as socket read and write.
+ * <p>
+ * For append, we process it as follow:
+ * <ol>
+ * <li>In the caller thread(typically, in the rpc handler thread):
+ * <ol>
+ * <li>Lock 'waitingConsumePayloads', bump nextTxid, and insert the entry to
+ * 'waitingConsumePayloads'.</li>
+ * <li>Schedule the consumer task if needed. See {@link #shouldScheduleConsumer()} for more details.
+ * </li>
+ * </ol>
+ * </li>
+ * <li>In the consumer task(in the EventLoop thread)
+ * <ol>
+ * <li>Poll the entry from 'waitingConsumePayloads' and insert it into 'waitingAppendEntries'</li>
+ * <li>Poll the entry from 'waitingAppendEntries', append it to the AsyncWriter, and insert it into
+ * 'unackedEntries'</li>
+ * <li>If the buffered size reaches {@link #batchSize}, or there is a sync request, then we call
+ * sync on the AsyncWriter.</li>
+ * <li>In the callback methods(CompletionHandler):
+ * <ul>
+ * <li>If succeeded, poll the entry from 'unackedEntries' and drop it.</li>
+ * <li>If failed, add all the entries in 'unackedEntries' back to 'waitingAppendEntries' and wait
+ * for writing them again.</li>
+ * </ul>
+ * </li>
+ * </ol>
+ * </li>
+ * </ol>
+ * For sync, the processing stages are almost same except that if it is not assigned with a new
+ * 'txid', we just assign the previous 'txid' to it without bumping the 'nextTxid'. And different
+ * from FSHLog, we will open a new writer and rewrite unacked entries to the new writer and sync
+ * again if we hit a sync error.
+ * <p>
+ * Here we only describe the logic of doReplaceWriter. The main logic of rollWriter is same with
+ * FSHLog.<br>
+ * For a normal roll request(for example, we have reached the log roll size):
+ * <ol>
+ * <li>In the log roller thread, we add a roll payload to 'waitingConsumePayloads', and then wait on
+ * the rollPromise(see {@link #waitForSafePoint()}).</li>
+ * <li>In the consumer thread, we will stop polling entries from 'waitingConsumePayloads' if we hit
+ * a Payload which contains a roll request.</li>
+ * <li>Append all entries to current writer, issue a sync request if possible.</li>
+ * <li>If sync succeeded, check if we could finish a roll request. There 3 conditions:
+ * <ul>
+ * <li>'rollPromise' is not null which means we have a pending roll request.</li>
+ * <li>'waitingAppendEntries' is empty.</li>
+ * <li>'unackedEntries' is empty.</li>
+ * </ul>
+ * </li>
+ * <li>Back to the log roller thread, now we can confirm that there are no out-going entries, i.e.,
+ * we reach a safe point. So it is safe to replace old writer with new writer now.</li>
+ * <li>Acquire 'waitingConsumePayloads' lock, set 'writerBroken' and 'waitingRoll' to false, cancel
+ * log roller exit checker if any(see the comments in the 'failed' method of the sync
+ * CompletionHandler to see why we need a checker here).</li>
+ * <li>Schedule the consumer task if needed.</li>
+ * <li>Schedule a background task to close the old writer.</li>
+ * </ol>
+ * For a broken writer roll request, the only difference is that we can bypass the wait for safe
+ * point stage. See the comments in the 'failed' method of the sync CompletionHandler for more
+ * details.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
+
+  private static final Log LOG = LogFactory.getLog(AsyncFSWAL.class);
+
+  public static final String WAL_BATCH_SIZE = "hbase.wal.batch.size";
+  public static final long DEFAULT_WAL_BATCH_SIZE = 64L * 1024;
+
+  public static final String ASYNC_WAL_CREATE_MAX_RETRIES = "hbase.wal.async.create.retries";
+  public static final int DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES = 10;
+
+  public static final String ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS =
+      "hbase.wal.async.logroller.exited.check.interval.ms";
+  public static final long DEFAULT_ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS = 1000;
+
+  /**
+   * Carry things that we want to pass to the consume task in event loop. Only one field can be
+   * non-null.
+   * <p>
+   * TODO: need to unify this and {@link RingBufferTruck}. There are mostly the same thing.
+   */
+  private static final class Payload {
+
+    // a wal entry which need to be appended
+    public final FSWALEntry entry;
+
+    // indicate that we need to sync our wal writer.
+    public final SyncFuture sync;
+
+    // incidate that we want to roll the writer.
+    public final Promise<Void> roll;
+
+    public Payload(FSWALEntry entry) {
+      this.entry = entry;
+      this.sync = null;
+      this.roll = null;
+    }
+
+    public Payload(SyncFuture sync) {
+      this.entry = null;
+      this.sync = sync;
+      this.roll = null;
+    }
+
+    public Payload(Promise<Void> roll) {
+      this.entry = null;
+      this.sync = null;
+      this.roll = roll;
+    }
+
+    @Override
+    public String toString() {
+      return "Payload [entry=" + entry + ", sync=" + sync + ", roll=" + roll + "]";
+    }
+  }
+
+  private final EventLoop eventLoop;
+
+  private final Deque<Payload> waitingConsumePayloads;
+
+  // like the ringbuffer sequence. Every FSWALEntry and SyncFuture will be assigned a txid and
+  // then added to waitingConsumePayloads.
+  private long nextTxid = 1L;
+
+  private boolean consumerScheduled;
+
+  // new writer is created and we are waiting for old writer to be closed.
+  private boolean waitingRoll;
+
+  // writer is broken and rollWriter is needed.
+  private boolean writerBroken;
+
+  private final long batchSize;
+
+  private final int createMaxRetries;
+
+  private final long logRollerExitedCheckIntervalMs;
+
+  private final ExecutorService closeExecutor = Executors
+      .newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true)
+          .setNameFormat("Close-WAL-Writer-%d").build());
+
+  private volatile FanOutOneBlockAsyncDFSOutput hdfsOut;
+
+  private final Deque<FSWALEntry> waitingAppendEntries = new ArrayDeque<FSWALEntry>();
+
+  private final Deque<FSWALEntry> unackedEntries = new ArrayDeque<FSWALEntry>();
+
+  private final PriorityQueue<SyncFuture> syncFutures = new PriorityQueue<SyncFuture>(11,
+      SEQ_COMPARATOR);
+
+  private Promise<Void> rollPromise;
+
+  // the highest txid of WAL entries being processed
+  private long highestProcessedTxid;
+
+  // file length when we issue last sync request on the writer
+  private long fileLengthAtLastSync;
+
+  private volatile boolean logRollerExited;
+
+  private final class LogRollerExitedChecker implements Runnable {
+
+    private boolean cancelled;
+
+    private ScheduledFuture<?> future;
+
+    public synchronized void setFuture(ScheduledFuture<?> future) {
+      this.future = future;
+    }
+
+    @Override
+    public void run() {
+      if (!logRollerExited) {
+        return;
+      }
+      // rollWriter is called in the log roller thread, and logRollerExited will be set just before
+      // the log rolled exit. So here we can confirm that no one could cancel us if the 'canceled'
+      // check passed. So it is safe to release the lock after checking 'canceled' flag.
+      synchronized (this) {
+        if (cancelled) {
+          return;
+        }
+      }
+      unackedEntries.clear();
+      waitingAppendEntries.clear();
+      IOException error = new IOException("sync failed but log roller exited");
+      for (SyncFuture future; (future = syncFutures.peek()) != null;) {
+        future.done(highestProcessedTxid, error);
+        syncFutures.remove();
+      }
+      synchronized (waitingConsumePayloads) {
+        for (Payload p : waitingConsumePayloads) {
+          if (p.entry != null) {
+            try {
+              p.entry.stampRegionSequenceId();
+            } catch (IOException e) {
+              throw new AssertionError("should not happen", e);
+            }
+          } else if (p.sync != null) {
+            p.sync.done(nextTxid, error);
+          }
+        }
+        waitingConsumePayloads.clear();
+      }
+    }
+
+    public synchronized void cancel() {
+      future.cancel(false);
+      cancelled = true;
+    }
+  }
+
+  private LogRollerExitedChecker logRollerExitedChecker;
+
+  public AsyncFSWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir,
+      Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
+      String prefix, String suffix, EventLoop eventLoop) throws FailedLogCloseException,
+      IOException {
+    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
+    this.eventLoop = eventLoop;
+    int maxHandlersCount = conf.getInt(REGION_SERVER_HANDLER_COUNT, 200);
+    waitingConsumePayloads = new ArrayDeque<Payload>(maxHandlersCount * 3);
+    batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
+    createMaxRetries =
+        conf.getInt(ASYNC_WAL_CREATE_MAX_RETRIES, DEFAULT_ASYNC_WAL_CREATE_MAX_RETRIES);
+    logRollerExitedCheckIntervalMs =
+        conf.getLong(ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS,
+          DEFAULT_ASYNC_WAL_LOG_ROLLER_EXITED_CHECK_INTERVAL_MS);
+    rollWriter();
+  }
+
+  private void tryFinishRoll() {
+    // 1. a roll is requested
+    // 2. we have written out all entries before the roll point.
+    // 3. all entries have been acked.
+    if (rollPromise != null && waitingAppendEntries.isEmpty() && unackedEntries.isEmpty()) {
+      rollPromise.trySuccess(null);
+      rollPromise = null;
+    }
+  }
+
+  private void sync(final AsyncWriter writer, final long processedTxid) {
+    fileLengthAtLastSync = writer.getLength();
+    final long startTimeNs = System.nanoTime();
+    writer.sync(new CompletionHandler<Long, Void>() {
+
+      @Override
+      public void completed(Long result, Void attachment) {
+        highestSyncedTxid.set(processedTxid);
+        int syncCount = finishSync(true);
+        for (Iterator<FSWALEntry> iter = unackedEntries.iterator(); iter.hasNext();) {
+          if (iter.next().getTxid() <= processedTxid) {
+            iter.remove();
+          } else {
+            break;
+          }
+        }
+        postSync(System.nanoTime() - startTimeNs, syncCount);
+        tryFinishRoll();
+        if (!rollWriterLock.tryLock()) {
+          return;
+        }
+        try {
+          if (writer.getLength() >= logrollsize) {
+            requestLogRoll();
+          }
+        } finally {
+          rollWriterLock.unlock();
+        }
+      }
+
+      @Override
+      public void failed(Throwable exc, Void attachment) {
+        LOG.warn("sync failed", exc);
+        // Here we depends on the implementation of FanOutOneBlockAsyncDFSOutput and netty.
+        // When error occur, FanOutOneBlockAsyncDFSOutput will fail all pending flush requests. It
+        // is execute inside EventLoop. And in DefaultPromise in netty, it will notifyListener
+        // directly if it is already in the EventLoop thread. And in the listener method, it will
+        // call us. So here we know that all failed flush request will call us continuously, and
+        // before the last one finish, no other task can be executed in EventLoop. So here we are
+        // safe to use writerBroken as a guard.
+        // Do not forget to revisit this if we change the implementation of
+        // FanOutOneBlockAsyncDFSOutput!
+        synchronized (waitingConsumePayloads) {
+          if (writerBroken) {
+            return;
+          }
+          // schedule a periodical task to check if log roller is exited. Otherwise the the sync
+          // request maybe blocked forever since we are still waiting for a new writer to write the
+          // pending data and sync it...
+          logRollerExitedChecker = new LogRollerExitedChecker();
+          // we are currently in the EventLoop thread, so it is safe to set the future after
+          // schedule it since the task can not be executed before we release the thread.
+          logRollerExitedChecker.setFuture(eventLoop.scheduleAtFixedRate(logRollerExitedChecker,
+            logRollerExitedCheckIntervalMs, logRollerExitedCheckIntervalMs, TimeUnit.MILLISECONDS));
+          writerBroken = true;
+        }
+        for (Iterator<FSWALEntry> iter = unackedEntries.descendingIterator(); iter.hasNext();) {
+          waitingAppendEntries.addFirst(iter.next());
+        }
+        highestUnsyncedTxid = highestSyncedTxid.get();
+        if (rollPromise != null) {
+          rollPromise.trySuccess(null);
+          rollPromise = null;
+          return;
+        }
+        // request a roll.
+        if (!rollWriterLock.tryLock()) {
+          return;
+        }
+        try {
+          requestLogRoll();
+        } finally {
+          rollWriterLock.unlock();
+        }
+      }
+    }, null);
+  }
+
+  private void addTimeAnnotation(SyncFuture future, String annotation) {
+    TraceScope scope = Trace.continueSpan(future.getSpan());
+    Trace.addTimelineAnnotation(annotation);
+    future.setSpan(scope.detach());
+  }
+
+  private int finishSync(boolean addSyncTrace) {
+    long doneTxid = highestSyncedTxid.get();
+    int finished = 0;
+    for (SyncFuture future; (future = syncFutures.peek()) != null;) {
+      if (future.getTxid() <= doneTxid) {
+        future.done(doneTxid, null);
+        syncFutures.remove();
+        finished++;
+        addTimeAnnotation(future, "writer synced");
+      } else {
+        break;
+      }
+    }
+    return finished;
+  }
+
+  private void consume() {
+    final AsyncWriter writer = this.writer;
+    // maybe a sync request is not queued when we issue a sync, so check here to see if we could
+    // finish some.
+    finishSync(false);
+    long newHighestProcessedTxid = -1L;
+    for (Iterator<FSWALEntry> iter = waitingAppendEntries.iterator(); iter.hasNext();) {
+      FSWALEntry entry = iter.next();
+      boolean appended;
+      try {
+        appended = append(writer, entry);
+      } catch (IOException e) {
+        throw new AssertionError("should not happen", e);
+      }
+      newHighestProcessedTxid = entry.getTxid();
+      iter.remove();
+      if (appended) {
+        unackedEntries.addLast(entry);
+        if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
+          break;
+        }
+      }
+    }
+    // if we have a newer transaction id, update it.
+    // otherwise, use the previous transaction id.
+    if (newHighestProcessedTxid > 0) {
+      highestProcessedTxid = newHighestProcessedTxid;
+    } else {
+      newHighestProcessedTxid = highestProcessedTxid;
+    }
+    if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
+      // sync because buffer size limit.
+      sync(writer, newHighestProcessedTxid);
+    } else if ((!syncFutures.isEmpty() || rollPromise != null)
+        && writer.getLength() > fileLengthAtLastSync) {
+      // first we should have at least one sync request or a roll request
+      // second we should have some unsynced data.
+      sync(writer, newHighestProcessedTxid);
+    } else if (writer.getLength() == fileLengthAtLastSync) {
+      // we haven't written anything out, just advance the highestSyncedSequence since we may only
+      // stamped some region sequence id.
+      highestSyncedTxid.set(newHighestProcessedTxid);
+      finishSync(false);
+      tryFinishRoll();
+    }
+  }
+
+  private static final Comparator<SyncFuture> SEQ_COMPARATOR = new Comparator<SyncFuture>() {
+
+    @Override
+    public int compare(SyncFuture o1, SyncFuture o2) {
+      int c = Long.compare(o1.getTxid(), o2.getTxid());
+      return c != 0 ? c : Integer.compare(System.identityHashCode(o1), System.identityHashCode(o2));
+    }
+  };
+
+  private final Runnable consumer = new Runnable() {
+
+    @Override
+    public void run() {
+      synchronized (waitingConsumePayloads) {
+        assert consumerScheduled;
+        if (writerBroken) {
+          // waiting for reschedule after rollWriter.
+          consumerScheduled = false;
+          return;
+        }
+        if (waitingRoll) {
+          // we may have toWriteEntries if the consume method does not write all pending entries
+          // out, this is usually happen if we have too many toWriteEntries that exceeded the
+          // batchSize limit.
+          if (waitingAppendEntries.isEmpty()) {
+            consumerScheduled = false;
+            return;
+          }
+        } else {
+          for (Payload p; (p = waitingConsumePayloads.pollFirst()) != null;) {
+            if (p.entry != null) {
+              waitingAppendEntries.addLast(p.entry);
+            } else if (p.sync != null) {
+              syncFutures.add(p.sync);
+            } else {
+              rollPromise = p.roll;
+              waitingRoll = true;
+              break;
+            }
+          }
+        }
+      }
+      consume();
+      synchronized (waitingConsumePayloads) {
+        if (waitingRoll) {
+          if (waitingAppendEntries.isEmpty()) {
+            consumerScheduled = false;
+            return;
+          }
+        } else {
+          if (waitingConsumePayloads.isEmpty() && waitingAppendEntries.isEmpty()) {
+            consumerScheduled = false;
+            return;
+          }
+        }
+      }
+      // reschedule if we still have something to write.
+      eventLoop.execute(this);
+    }
+  };
+
+  private boolean shouldScheduleConsumer() {
+    if (writerBroken || waitingRoll) {
+      return false;
+    }
+    if (consumerScheduled) {
+      return false;
+    }
+    consumerScheduled = true;
+    return true;
+  }
+
+  @Override
+  public long append(HRegionInfo hri, WALKey key, WALEdit edits, boolean inMemstore)
+      throws IOException {
+    boolean scheduleTask;
+    long txid;
+    synchronized (waitingConsumePayloads) {
+      if (this.closed) {
+        throw new IOException("Cannot append; log is closed");
+      }
+      txid = nextTxid++;
+      FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore);
+      scheduleTask = shouldScheduleConsumer();
+      waitingConsumePayloads.add(new Payload(entry));
+    }
+    if (scheduleTask) {
+      eventLoop.execute(consumer);
+    }
+    return txid;
+  }
+
+  @Override
+  public void sync() throws IOException {
+    TraceScope scope = Trace.startSpan("AsyncFSWAL.sync");
+    try {
+      SyncFuture future;
+      boolean scheduleTask;
+      synchronized (waitingConsumePayloads) {
+        scheduleTask = shouldScheduleConsumer();
+        future = getSyncFuture(nextTxid - 1, scope.detach());
+        waitingConsumePayloads.addLast(new Payload(future));
+      }
+      if (scheduleTask) {
+        eventLoop.execute(consumer);
+      }
+      scope = Trace.continueSpan(blockOnSync(future));
+    } finally {
+      assert scope == NullScope.INSTANCE || !scope.isDetached();
+      scope.close();
+    }
+  }
+
+  @Override
+  public void sync(long txid) throws IOException {
+    if (highestSyncedTxid.get() >= txid) {
+      return;
+    }
+    TraceScope scope = Trace.startSpan("AsyncFSWAL.sync");
+    try {
+      SyncFuture future = getSyncFuture(txid, scope.detach());
+      boolean scheduleTask;
+      synchronized (waitingConsumePayloads) {
+        scheduleTask = shouldScheduleConsumer();
+        waitingConsumePayloads.addLast(new Payload(future));
+      }
+      if (scheduleTask) {
+        eventLoop.execute(consumer);
+      }
+      scope = Trace.continueSpan(blockOnSync(future));
+    } finally {
+      assert scope == NullScope.INSTANCE || !scope.isDetached();
+      scope.close();
+    }
+  }
+
+  @Override
+  public void logRollerExited() {
+    logRollerExited = true;
+  }
+
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    boolean overwrite = false;
+    for (int retry = 0;; retry++) {
+      try {
+        return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, overwrite, eventLoop);
+      } catch (RemoteException e) {
+        LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
+        if (shouldRetryCreate(e)) {
+          if (retry >= createMaxRetries) {
+            break;
+          }
+        } else {
+          throw e.unwrapRemoteException();
+        }
+      } catch (NameNodeException e) {
+        throw e;
+      } catch (IOException e) {
+        LOG.warn("create wal log writer " + path + " failed, retry = " + retry, e);
+        if (retry >= createMaxRetries) {
+          break;
+        }
+        // overwrite the old broken file.
+        overwrite = true;
+        try {
+          Thread.sleep(ConnectionUtils.getPauseTime(100, retry));
+        } catch (InterruptedException ie) {
+          throw new InterruptedIOException();
+        }
+      }
+    }
+    throw new IOException("Failed to create wal log writer " + path + " after retrying "
+        + createMaxRetries + " time(s)");
+  }
+
+  private void waitForSafePoint() {
+    Future<Void> roll;
+    boolean scheduleTask;
+    synchronized (waitingConsumePayloads) {
+      if (!writerBroken && this.writer != null) {
+        Promise<Void> promise = eventLoop.newPromise();
+        if (consumerScheduled) {
+          scheduleTask = false;
+        } else {
+          scheduleTask = consumerScheduled = true;
+        }
+        waitingConsumePayloads.addLast(new Payload(promise));
+        roll = promise;
+      } else {
+        roll = eventLoop.newSucceededFuture(null);
+        scheduleTask = false;
+      }
+    }
+    if (scheduleTask) {
+      eventLoop.execute(consumer);
+    }
+    roll.awaitUninterruptibly();
+  }
+
+  @Override
+  protected long doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter)
+      throws IOException {
+    waitForSafePoint();
+    final AsyncWriter oldWriter = this.writer;
+    this.writer = nextWriter;
+    if (nextWriter != null && nextWriter instanceof AsyncProtobufLogWriter) {
+      this.hdfsOut = ((AsyncProtobufLogWriter) nextWriter).getOutput();
+    }
+    this.fileLengthAtLastSync = 0L;
+    boolean scheduleTask;
+    synchronized (waitingConsumePayloads) {
+      writerBroken = waitingRoll = false;
+      if (logRollerExitedChecker != null) {
+        logRollerExitedChecker.cancel();
+        logRollerExitedChecker = null;
+      }
+      if (consumerScheduled) {
+        scheduleTask = false;
+      } else {
+        if (waitingConsumePayloads.isEmpty() && waitingAppendEntries.isEmpty()) {
+          scheduleTask = false;
+        } else {
+          scheduleTask = consumerScheduled = true;
+        }
+      }
+    }
+    if (scheduleTask) {
+      eventLoop.execute(consumer);
+    }
+    long oldFileLen;
+    if (oldWriter != null) {
+      oldFileLen = oldWriter.getLength();
+      closeExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            oldWriter.close();
+          } catch (IOException e) {
+            LOG.warn("close old writer failed", e);
+          }
+        }
+      });
+    } else {
+      oldFileLen = 0L;
+    }
+    return oldFileLen;
+  }
+
+  @Override
+  protected void doShutdown() throws IOException {
+    waitForSafePoint();
+    this.writer.close();
+    this.writer = null;
+    closeExecutor.shutdown();
+  }
+
+  @Override
+  protected void doAppend(AsyncWriter writer, FSWALEntry entry) {
+    writer.append(entry);
+  }
+
+  @Override
+  DatanodeInfo[] getPipeline() {
+    FanOutOneBlockAsyncDFSOutput output = this.hdfsOut;
+    return output != null ? output.getPipeline() : new DatanodeInfo[0];
+  }
+
+  @Override
+  int getLogReplication() {
+    return getPipeline().length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
new file mode 100644
index 0000000..894f3dd
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.nio.channels.CompletionHandler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutput;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+
+import com.google.common.base.Throwables;
+import com.google.common.primitives.Ints;
+
+import io.netty.channel.EventLoop;
+
+/**
+ * AsyncWriter for protobuf-based WAL.
+ */
+@InterfaceAudience.Private
+public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter implements
+    AsyncFSWALProvider.AsyncWriter {
+
+  private static final Log LOG = LogFactory.getLog(AsyncProtobufLogWriter.class);
+
+  private static final class BlockingCompletionHandler implements CompletionHandler<Long, Void> {
+
+    private long size;
+
+    private Throwable error;
+
+    private boolean finished;
+
+    @Override
+    public void completed(Long result, Void attachment) {
+      synchronized (this) {
+        size = result.longValue();
+        finished = true;
+        notifyAll();
+      }
+    }
+
+    @Override
+    public void failed(Throwable exc, Void attachment) {
+      synchronized (this) {
+        error = exc;
+        finished = true;
+        notifyAll();
+      }
+    }
+
+    public long get() throws IOException {
+      synchronized (this) {
+        while (!finished) {
+          try {
+            wait();
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException();
+          }
+        }
+        if (error != null) {
+          Throwables.propagateIfPossible(error, IOException.class);
+          throw new RuntimeException(error);
+        }
+        return size;
+      }
+    }
+  }
+
+  private final EventLoop eventLoop;
+
+  private FanOutOneBlockAsyncDFSOutput output;
+
+  private ByteArrayOutputStream buf;
+
+  public AsyncProtobufLogWriter(EventLoop eventLoop) {
+    this.eventLoop = eventLoop;
+  }
+
+  @Override
+  public void append(Entry entry) {
+    buf.reset();
+    entry.setCompressionContext(compressionContext);
+    try {
+      entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
+          .writeDelimitedTo(buf);
+    } catch (IOException e) {
+      throw new AssertionError("should not happen", e);
+    }
+    length.addAndGet(buf.size());
+    output.write(buf.getBuffer(), 0, buf.size());
+    try {
+      for (Cell cell : entry.getEdit().getCells()) {
+        buf.reset();
+        cellEncoder.write(cell);
+        length.addAndGet(buf.size());
+        output.write(buf.getBuffer(), 0, buf.size());
+      }
+    } catch (IOException e) {
+      throw new AssertionError("should not happen", e);
+    }
+  }
+
+  @Override
+  public <A> void sync(CompletionHandler<Long, A> handler, A attachment) {
+    output.flush(attachment, handler, false);
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (this.output == null) {
+      return;
+    }
+    try {
+      writeWALTrailer();
+      output.close();
+    } catch (Exception e) {
+      LOG.warn("normal close failed, try recover", e);
+      output.recoverAndClose(null);
+    }
+    this.output = null;
+  }
+
+  public FanOutOneBlockAsyncDFSOutput getOutput() {
+    return this.output;
+  }
+
+  @Override
+  protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
+      short replication, long blockSize) throws IOException {
+    this.output =
+        FanOutOneBlockAsyncDFSOutputHelper.createOutput((DistributedFileSystem) fs, path,
+          overwritable, false, replication, blockSize, eventLoop);
+    this.buf = new ByteArrayOutputStream();
+  }
+
+  @Override
+  protected long writeMagicAndWALHeader(byte[] magic, WALHeader header) throws IOException {
+    buf.reset();
+    header.writeDelimitedTo(buf);
+    final BlockingCompletionHandler handler = new BlockingCompletionHandler();
+    eventLoop.execute(new Runnable() {
+
+      @Override
+      public void run() {
+        output.write(ProtobufLogReader.PB_WAL_MAGIC);
+        output.write(buf.getBuffer(), 0, buf.size());
+        output.flush(null, handler, false);
+      }
+    });
+    return handler.get();
+  }
+
+  @Override
+  protected long writeWALTrailerAndMagic(WALTrailer trailer, final byte[] magic)
+      throws IOException {
+    buf.reset();
+    trailer.writeTo(buf);
+    final BlockingCompletionHandler handler = new BlockingCompletionHandler();
+    eventLoop.execute(new Runnable() {
+      public void run() {
+        output.write(buf.getBuffer(), 0, buf.size());
+        output.write(Ints.toByteArray(buf.size()));
+        output.write(magic);
+        output.flush(null, handler, false);
+      }
+    });
+    return handler.get();
+  }
+
+  @Override
+  protected OutputStream getOutputStreamForCellEncoder() {
+    return buf;
+  }
+}


[3/4] hbase git commit: HBASE-15265 Implement an asynchronous FSHLog

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/c96b642f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
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 dfbdae5..67c2b93 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
@@ -17,62 +17,44 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.apache.hadoop.hbase.wal.DefaultWALProvider.WAL_FILE_NAME_DELIMITER;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.io.OutputStream;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryUsage;
-import java.lang.reflect.InvocationTargetException;
-import java.net.URLEncoder;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Set;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.ExceptionHandler;
+import com.lmax.disruptor.LifecycleAware;
+import com.lmax.disruptor.TimeoutException;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
-import org.apache.hadoop.hbase.util.DrainBarrier;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
@@ -81,95 +63,59 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.NullScope;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.lmax.disruptor.BlockingWaitStrategy;
-import com.lmax.disruptor.EventHandler;
-import com.lmax.disruptor.ExceptionHandler;
-import com.lmax.disruptor.LifecycleAware;
-import com.lmax.disruptor.TimeoutException;
-import com.lmax.disruptor.dsl.Disruptor;
-import com.lmax.disruptor.dsl.ProducerType;
-
 /**
- * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS.
- * Only one WAL is ever being written at a time.  When a WAL hits a configured maximum size,
- * it is rolled.  This is done internal to the implementation.
- *
- * <p>As data is flushed from the MemStore to other on-disk structures (files sorted by
- * key, hfiles), a WAL becomes obsolete. We can let go of all the log edits/entries for a given
- * HRegion-sequence id.  A bunch of work in the below is done keeping account of these region
- * sequence ids -- what is flushed out to hfiles, and what is yet in WAL and in memory only.
- *
- * <p>It is only practical to delete entire files. Thus, we delete an entire on-disk file
- * <code>F</code> when all of the edits in <code>F</code> have a log-sequence-id that's older
- * (smaller) than the most-recent flush.
- *
- * <p>To read an WAL, call {@link WALFactory#createReader(org.apache.hadoop.fs.FileSystem,
- * org.apache.hadoop.fs.Path)}.
- *
- * <h2>Failure Semantic</h2>
- * If an exception on append or sync, roll the WAL because the current WAL is now a lame duck;
- * any more appends or syncs will fail also with the same original exception. If we have made
- * successful appends to the WAL and we then are unable to sync them, our current semantic is to
- * return error to the client that the appends failed but also to abort the current context,
- * usually the hosting server. We need to replay the WALs. TODO: Change this semantic. A roll of
- * WAL may be sufficient as long as we have flagged client that the append failed. TODO:
- * replication may pick up these last edits though they have been marked as failed append (Need to
- * keep our own file lengths, not rely on HDFS).
+ * The default implementation of FSWAL.
  */
 @InterfaceAudience.Private
-public class FSHLog implements WAL {
+public class FSHLog extends AbstractFSWAL<Writer> {
   // IMPLEMENTATION NOTES:
   //
-  // At the core is a ring buffer.  Our ring buffer is the LMAX Disruptor.  It tries to
+  // At the core is a ring buffer. Our ring buffer is the LMAX Disruptor. It tries to
   // minimize synchronizations and volatile writes when multiple contending threads as is the case
-  // here appending and syncing on a single WAL.  The Disruptor is configured to handle multiple
+  // here appending and syncing on a single WAL. The Disruptor is configured to handle multiple
   // producers but it has one consumer only (the producers in HBase are IPC Handlers calling append
-  // and then sync).  The single consumer/writer pulls the appends and syncs off the ring buffer.
+  // and then sync). The single consumer/writer pulls the appends and syncs off the ring buffer.
   // When a handler calls sync, it is given back a future. The producer 'blocks' on the future so
-  // it does not return until the sync completes.  The future is passed over the ring buffer from
+  // it does not return until the sync completes. The future is passed over the ring buffer from
   // the producer/handler to the consumer thread where it does its best to batch up the producer
-  // syncs so one WAL sync actually spans multiple producer sync invocations.  How well the
+  // syncs so one WAL sync actually spans multiple producer sync invocations. How well the
   // batching works depends on the write rate; i.e. we tend to batch more in times of
   // high writes/syncs.
   //
   // Calls to append now also wait until the append has been done on the consumer side of the
-  // disruptor.  We used to not wait but it makes the implemenation easier to grok if we have
+  // disruptor. We used to not wait but it makes the implementation easier to grok if we have
   // the region edit/sequence id after the append returns.
   //
-  // TODO: Handlers need to coordinate appending AND syncing.  Can we have the threads contend
-  // once only?  Probably hard given syncs take way longer than an append.
+  // TODO: Handlers need to coordinate appending AND syncing. Can we have the threads contend
+  // once only? Probably hard given syncs take way longer than an append.
   //
   // The consumer threads pass the syncs off to multiple syncing threads in a round robin fashion
   // to ensure we keep up back-to-back FS sync calls (FS sync calls are the long poll writing the
-  // WAL).  The consumer thread passes the futures to the sync threads for it to complete
+  // WAL). The consumer thread passes the futures to the sync threads for it to complete
   // the futures when done.
   //
-  // The 'sequence' in the below is the sequence of the append/sync on the ringbuffer.  It
-  // acts as a sort-of transaction id.  It is always incrementing.
+  // The 'sequence' in the below is the sequence of the append/sync on the ringbuffer. It
+  // acts as a sort-of transaction id. It is always incrementing.
   //
-  // The RingBufferEventHandler class hosts the ring buffer consuming code.  The threads that
-  // do the actual FS sync are implementations of SyncRunner.  SafePointZigZagLatch is a
-  // synchronization class used to halt the consumer at a safe point --  just after all outstanding
+  // The RingBufferEventHandler class hosts the ring buffer consuming code. The threads that
+  // do the actual FS sync are implementations of SyncRunner. SafePointZigZagLatch is a
+  // synchronization class used to halt the consumer at a safe point -- just after all outstanding
   // syncs and appends have completed -- so the log roller can swap the WAL out under it.
-
+  //
+  // We use ring buffer sequence as txid of FSWALEntry and SyncFuture.
   private static final Log LOG = LogFactory.getLog(FSHLog.class);
 
-  private static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
-
   /**
-   * The nexus at which all incoming handlers meet.  Does appends and sync with an ordering.
-   * Appends and syncs are each put on the ring which means handlers need to
-   * smash up against the ring twice (can we make it once only? ... maybe not since time to append
-   * is so different from time to sync and sometimes we don't want to sync or we want to async
-   * the sync).  The ring is where we make sure of our ordering and it is also where we do
-   * batching up of handler sync calls.
+   * The nexus at which all incoming handlers meet. Does appends and sync with an ordering. Appends
+   * and syncs are each put on the ring which means handlers need to smash up against the ring twice
+   * (can we make it once only? ... maybe not since time to append is so different from time to sync
+   * and sometimes we don't want to sync or we want to async the sync). The ring is where we make
+   * sure of our ordering and it is also where we do batching up of handler sync calls.
    */
   private final Disruptor<RingBufferTruck> disruptor;
 
@@ -180,95 +126,13 @@ public class FSHLog implements WAL {
 
   /**
    * This fellow is run by the above appendExecutor service but it is all about batching up appends
-   * and syncs; it may shutdown without cleaning out the last few appends or syncs.  To guard
-   * against this, keep a reference to this handler and do explicit close on way out to make sure
-   * all flushed out before we exit.
+   * and syncs; it may shutdown without cleaning out the last few appends or syncs. To guard against
+   * this, keep a reference to this handler and do explicit close on way out to make sure all
+   * flushed out before we exit.
    */
   private final RingBufferEventHandler ringBufferEventHandler;
 
   /**
-   * Map of {@link SyncFuture}s keyed by Handler objects.  Used so we reuse SyncFutures.
-   * TODO: Reus FSWALEntry's rather than create them anew each time as we do SyncFutures here.
-   * TODO: Add a FSWalEntry and SyncFuture as thread locals on handlers rather than have them
-   * get them from this Map?
-   */
-  private final Map<Thread, SyncFuture> syncFuturesByHandler;
-
-  /**
-   * The highest known outstanding unsync'd WALEdit sequence number where sequence number is the
-   * ring buffer sequence.  Maintained by the ring buffer consumer.
-   */
-  private volatile long highestUnsyncedSequence = -1;
-
-  /**
-   * Updated to the ring buffer sequence of the last successful sync call.  This can be less than
-   * {@link #highestUnsyncedSequence} for case where we have an append where a sync has not yet
-   * come in for it.  Maintained by the syncing threads.
-   */
-  private final AtomicLong highestSyncedSequence = new AtomicLong(0);
-
-  /**
-   * file system instance
-   */
-  protected final FileSystem fs;
-
-  /**
-   * WAL directory, where all WAL files would be placed.
-   */
-  private final Path fullPathLogDir;
-
-  /**
-   * dir path where old logs are kept.
-   */
-  private final Path fullPathArchiveDir;
-
-  /**
-   * Matches just those wal files that belong to this wal instance.
-   */
-  private final PathFilter ourFiles;
-
-  /**
-   * Prefix of a WAL file, usually the region server name it is hosted on.
-   */
-  private final String logFilePrefix;
-
-  /**
-   * Suffix included on generated wal file names
-   */
-  private final String logFileSuffix;
-
-  /**
-   * Prefix used when checking for wal membership.
-   */
-  private final String prefixPathStr;
-
-  private final WALCoprocessorHost coprocessorHost;
-
-  /**
-   * conf object
-   */
-  protected final Configuration conf;
-
-  /** Listeners that are called on WAL events. */
-  private final List<WALActionsListener> listeners =
-    new CopyOnWriteArrayList<WALActionsListener>();
-
-  @Override
-  public void registerWALActionsListener(final WALActionsListener listener) {
-    this.listeners.add(listener);
-  }
-
-  @Override
-  public boolean unregisterWALActionsListener(final WALActionsListener listener) {
-    return this.listeners.remove(listener);
-  }
-
-  @Override
-  public WALCoprocessorHost getCoprocessorHost() {
-    return coprocessorHost;
-  }
-
-  /**
    * FSDataOutputStream associated with the current SequenceFile.writer
    */
   private FSDataOutputStream hdfs_out;
@@ -278,8 +142,6 @@ public class FSHLog implements WAL {
   // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered
   private final int minTolerableReplication;
 
-  private final int slowSyncNs;
-
   // 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
@@ -293,84 +155,14 @@ public class FSHLog implements WAL {
   // Enable it if the replications recover.
   private volatile boolean lowReplicationRollEnabled = true;
 
-  /**
-   * Class that does accounting of sequenceids in WAL subsystem. Holds oldest outstanding
-   * sequence id as yet not flushed as well as the most recent edit sequence id appended to the
-   * WAL. Has facility for answering questions such as "Is it safe to GC a WAL?".
-   */
-  private SequenceIdAccounting sequenceIdAccounting = new SequenceIdAccounting();
-
-  /**
-   * Current log file.
-   */
-  volatile Writer writer;
-
-  /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */
-  private final DrainBarrier closeBarrier = new DrainBarrier();
-
-  /**
-   * This lock makes sure only one log roll runs at a time. Should not be taken while any other
-   * lock is held. We don't just use synchronized because that results in bogus and tedious
-   * findbugs warning when it thinks synchronized controls writer thread safety.  It is held when
-   * we are actually rolling the log.  It is checked when we are looking to see if we should roll
-   * the log or not.
-   */
-  private final ReentrantLock rollWriterLock = new ReentrantLock(true);
-
-  private volatile boolean closed = false;
-  private final AtomicBoolean shutdown = new AtomicBoolean(false);
-
-  // The timestamp (in ms) when the log file was created.
-  private final AtomicLong filenum = new AtomicLong(-1);
-
-  // Number of transactions in the current Wal.
-  private final AtomicInteger numEntries = new AtomicInteger(0);
-
-  // If > than this size, roll the log.
-  private final long logrollsize;
-
-  /**
-   * The total size of wal
-   */
-  private AtomicLong totalLogSize = new AtomicLong(0);
-
-  /*
-   * If more than this many logs, force flush of oldest region to oldest edit
-   * goes to disk.  If too many and we crash, then will take forever replaying.
-   * Keep the number of logs tidy.
-   */
-  private final int maxLogs;
-
   /** Number of log close errors tolerated before we abort */
   private final int closeErrorsTolerated;
 
   private final AtomicInteger closeErrorCount = new AtomicInteger();
 
-
   /**
-   * WAL Comparator; it compares the timestamp (log filenum), present in the log file name.
-   * Throws an IllegalArgumentException if used to compare paths from different wals.
-   */
-  final Comparator<Path> LOG_NAME_COMPARATOR = new Comparator<Path>() {
-    @Override
-    public int compare(Path o1, Path o2) {
-      long t1 = getFileNumFromFileName(o1);
-      long t2 = getFileNumFromFileName(o2);
-      if (t1 == t2) return 0;
-      return (t1 > t2) ? 1 : -1;
-    }
-  };
-
-  /**
-   * Map of WAL log file to the latest sequence ids of all regions it has entries of.
-   * The map is sorted by the log file creation timestamp (contained in the log file name).
-   */
-  private NavigableMap<Path, Map<byte[], Long>> byWalRegionSequenceIds =
-    new ConcurrentSkipListMap<Path, Map<byte[], Long>>(LOG_NAME_COMPARATOR);
-
-  /**
-   * Exception handler to pass the disruptor ringbuffer.  Same as native implementation only it
-   * logs using our logger instead of java native logger.
+   * Exception handler to pass the disruptor ringbuffer. Same as native implementation only it logs
+   * using our logger instead of java native logger.
    */
   static class RingBufferExceptionHandler implements ExceptionHandler {
     @Override
@@ -394,12 +186,10 @@ public class FSHLog implements WAL {
 
   /**
    * Constructor.
-   *
    * @param fs filesystem handle
    * @param root path for stored and archived wals
    * @param logDir dir where wals are stored
    * @param conf configuration to use
-   * @throws IOException
    */
   public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)
       throws IOException {
@@ -407,252 +197,79 @@ public class FSHLog implements WAL {
   }
 
   /**
-   * Create an edit log at the given <code>dir</code> location.
-   *
-   * You should never have to load an existing log. If there is a log at
-   * startup, it should have already been processed and deleted by the time the
-   * WAL object is started up.
-   *
+   * Create an edit log at the given <code>dir</code> location. You should never have to load an
+   * existing log. If there is a log at startup, it should have already been processed and deleted
+   * by the time the WAL object is started up.
    * @param fs filesystem handle
    * @param rootDir path to where logs and oldlogs
    * @param logDir dir where wals are stored
    * @param archiveDir dir where wals are archived
    * @param conf configuration to use
-   * @param listeners Listeners on WAL events. Listeners passed here will
-   * be registered before we do anything else; e.g. the
-   * Constructor {@link #rollWriter()}.
-   * @param failIfWALExists If true IOException will be thrown if files related to this wal
-   *        already exist.
-   * @param prefix should always be hostname and port in distributed env and
-   *        it will be URL encoded before being used.
-   *        If prefix is null, "wal" will be used
+   * @param listeners Listeners on WAL events. Listeners passed here will be registered before we do
+   *          anything else; e.g. the Constructor {@link #rollWriter()}.
+   * @param failIfWALExists If true IOException will be thrown if files related to this wal already
+   *          exist.
+   * @param prefix should always be hostname and port in distributed env and it will be URL encoded
+   *          before being used. If prefix is null, "wal" will be used
    * @param suffix will be url encoded. null is treated as empty. non-empty must start with
-   *        {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
-   * @throws IOException
+   *          {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
    */
   public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
-      final String archiveDir, final Configuration conf,
-      final List<WALActionsListener> listeners,
-      final boolean failIfWALExists, final String prefix, final String suffix)
-      throws IOException {
-    this.fs = fs;
-    this.fullPathLogDir = new Path(rootDir, logDir);
-    this.fullPathArchiveDir = new Path(rootDir, archiveDir);
-    this.conf = conf;
-
-    if (!fs.exists(fullPathLogDir) && !fs.mkdirs(fullPathLogDir)) {
-      throw new IOException("Unable to mkdir " + fullPathLogDir);
-    }
-
-    if (!fs.exists(this.fullPathArchiveDir)) {
-      if (!fs.mkdirs(this.fullPathArchiveDir)) {
-        throw new IOException("Unable to mkdir " + this.fullPathArchiveDir);
-      }
-    }
-
-    // If prefix is null||empty then just name it wal
-    this.logFilePrefix =
-      prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8");
-    // we only correctly differentiate suffices when numeric ones start with '.'
-    if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) {
-      throw new IllegalArgumentException("WAL suffix must start with '" + WAL_FILE_NAME_DELIMITER +
-          "' but instead was '" + suffix + "'");
-    }
-    // Now that it exists, set the storage policy for the entire directory of wal files related to
-    // this FSHLog instance
-    FSUtils.setStoragePolicy(fs, conf, this.fullPathLogDir, HConstants.WAL_STORAGE_POLICY,
-      HConstants.DEFAULT_WAL_STORAGE_POLICY);
-    this.logFileSuffix = (suffix == null) ? "" : URLEncoder.encode(suffix, "UTF8");
-    this.prefixPathStr = new Path(fullPathLogDir,
-        logFilePrefix + WAL_FILE_NAME_DELIMITER).toString();
-
-    this.ourFiles = new PathFilter() {
-      @Override
-      public boolean accept(final Path fileName) {
-        // The path should start with dir/<prefix> and end with our suffix
-        final String fileNameString = fileName.toString();
-        if (!fileNameString.startsWith(prefixPathStr)) {
-          return false;
-        }
-        if (logFileSuffix.isEmpty()) {
-          // in the case of the null suffix, we need to ensure the filename ends with a timestamp.
-          return org.apache.commons.lang.StringUtils.isNumeric(
-              fileNameString.substring(prefixPathStr.length()));
-        } else if (!fileNameString.endsWith(logFileSuffix)) {
-          return false;
-        }
-        return true;
-      }
-    };
-
-    if (failIfWALExists) {
-      final FileStatus[] walFiles = FSUtils.listStatus(fs, fullPathLogDir, ourFiles);
-      if (null != walFiles && 0 != walFiles.length) {
-        throw new IOException("Target WAL already exists within directory " + fullPathLogDir);
-      }
-    }
-
-    // Register listeners.  TODO: Should this exist anymore?  We have CPs?
-    if (listeners != null) {
-      for (WALActionsListener i: listeners) {
-        registerWALActionsListener(i);
-      }
-    }
-    this.coprocessorHost = new WALCoprocessorHost(this, conf);
-
-    // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks
-    // (it costs a little x'ing bocks)
-    final long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize",
-        FSUtils.getDefaultBlockSize(this.fs, this.fullPathLogDir));
-    this.logrollsize =
-      (long)(blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
-
-    float memstoreRatio = conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY,
-      conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_OLD_KEY,
-        HeapMemorySizeUtil.DEFAULT_MEMSTORE_SIZE));
-    boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null;
-    if(maxLogsDefined){
-      LOG.warn("'hbase.regionserver.maxlogs' was deprecated.");
-    }
-    this.maxLogs = conf.getInt("hbase.regionserver.maxlogs",
-        Math.max(32, calculateMaxLogFiles(memstoreRatio, logrollsize)));
+      final String archiveDir, final Configuration conf, final List<WALActionsListener> listeners,
+      final boolean failIfWALExists, final String prefix, final String suffix) throws IOException {
+    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
     this.minTolerableReplication = conf.getInt("hbase.regionserver.hlog.tolerable.lowreplication",
-        FSUtils.getDefaultReplication(fs, this.fullPathLogDir));
-    this.lowReplicationRollLimit =
-      conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit", 5);
+      FSUtils.getDefaultReplication(fs, this.walDir));
+    this.lowReplicationRollLimit = conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit",
+      5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 0);
-    int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
-
-    LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) +
-      ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
-      ", prefix=" + this.logFilePrefix + ", suffix=" + logFileSuffix + ", logDir=" +
-      this.fullPathLogDir + ", archiveDir=" + this.fullPathArchiveDir);
 
     // rollWriter sets this.hdfs_out if it can.
     rollWriter();
 
-    this.slowSyncNs =
-        1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms",
-          DEFAULT_SLOW_SYNC_TIME_MS);
-
-    // This is the 'writer' -- a single threaded executor.  This single thread 'consumes' what is
+    // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is
     // put on the ring buffer.
     String hostingThreadName = Thread.currentThread().getName();
-    this.appendExecutor = Executors.
-      newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName + ".append"));
-    // Preallocate objects to use on the ring buffer.  The way that appends and syncs work, we will
+    this.appendExecutor = Executors
+        .newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName + ".append"));
+    // Preallocate objects to use on the ring buffer. The way that appends and syncs work, we will
     // be stuck and make no progress if the buffer is filled with appends only and there is no
     // sync. If no sync, then the handlers will be outstanding just waiting on sync completion
     // before they return.
-    final int preallocatedEventCount =
-      this.conf.getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16);
-    // Using BlockingWaitStrategy.  Stuff that is going on here takes so long it makes no sense
+    final int preallocatedEventCount = this.conf
+        .getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16);
+    // Using BlockingWaitStrategy. Stuff that is going on here takes so long it makes no sense
     // spinning as other strategies do.
-    this.disruptor =
-      new Disruptor<RingBufferTruck>(RingBufferTruck.EVENT_FACTORY, preallocatedEventCount,
-        this.appendExecutor, ProducerType.MULTI, new BlockingWaitStrategy());
+    this.disruptor = new Disruptor<RingBufferTruck>(RingBufferTruck.EVENT_FACTORY,
+        preallocatedEventCount, this.appendExecutor, ProducerType.MULTI,
+        new BlockingWaitStrategy());
     // Advance the ring buffer sequence so that it starts from 1 instead of 0,
     // because SyncFuture.NOT_DONE = 0.
     this.disruptor.getRingBuffer().next();
-    this.ringBufferEventHandler =
-      new RingBufferEventHandler(conf.getInt("hbase.regionserver.hlog.syncer.count", 5),
-        maxHandlersCount);
+    int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
+    this.ringBufferEventHandler = new RingBufferEventHandler(
+        conf.getInt("hbase.regionserver.hlog.syncer.count", 5), maxHandlersCount);
     this.disruptor.handleExceptionsWith(new RingBufferExceptionHandler());
-    this.disruptor.handleEventsWith(new RingBufferEventHandler [] {this.ringBufferEventHandler});
-    // Presize our map of SyncFutures by handler objects.
-    this.syncFuturesByHandler = new ConcurrentHashMap<Thread, SyncFuture>(maxHandlersCount);
+    this.disruptor.handleEventsWith(new RingBufferEventHandler[] { this.ringBufferEventHandler });
     // Starting up threads in constructor is a no no; Interface should have an init call.
     this.disruptor.start();
   }
 
-  private int calculateMaxLogFiles(float memstoreSizeRatio, long logRollSize) {
-    MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
-    int maxLogs = Math.round(mu.getMax() * memstoreSizeRatio * 2 / logRollSize);
-    return maxLogs;
-  }
-
-  /**
-   * Get the backing files associated with this WAL.
-   * @return may be null if there are no files.
-   */
-  protected FileStatus[] getFiles() throws IOException {
-    return FSUtils.listStatus(fs, fullPathLogDir, ourFiles);
-  }
-
   /**
-   * Currently, we need to expose the writer's OutputStream to tests so that they can manipulate
-   * the default behavior (such as setting the maxRecoveryErrorCount value for example (see
+   * Currently, we need to expose the writer's OutputStream to tests so that they can manipulate the
+   * default behavior (such as setting the maxRecoveryErrorCount value for example (see
    * {@link TestWALReplay#testReplayEditsWrittenIntoWAL()}). This is done using reflection on the
-   * underlying HDFS OutputStream.
-   * NOTE: This could be removed once Hadoop1 support is removed.
+   * underlying HDFS OutputStream. NOTE: This could be removed once Hadoop1 support is removed.
    * @return null if underlying stream is not ready.
    */
   @VisibleForTesting
   OutputStream getOutputStream() {
     FSDataOutputStream fsdos = this.hdfs_out;
-    if (fsdos == null) return null;
-    return fsdos.getWrappedStream();
-  }
-
-  @Override
-  public byte [][] rollWriter() throws FailedLogCloseException, IOException {
-    return rollWriter(false);
-  }
-
-  /**
-   * retrieve the next path to use for writing.
-   * Increments the internal filenum.
-   */
-  private Path getNewPath() throws IOException {
-    this.filenum.set(System.currentTimeMillis());
-    Path newPath = getCurrentFileName();
-    while (fs.exists(newPath)) {
-      this.filenum.incrementAndGet();
-      newPath = getCurrentFileName();
-    }
-    return newPath;
-  }
-
-  Path getOldPath() {
-    long currentFilenum = this.filenum.get();
-    Path oldPath = null;
-    if (currentFilenum > 0) {
-      // ComputeFilename  will take care of meta wal filename
-      oldPath = computeFilename(currentFilenum);
-    } // I presume if currentFilenum is <= 0, this is first file and null for oldPath if fine?
-    return oldPath;
-  }
-
-  /**
-   * Tell listeners about pre log roll.
-   * @throws IOException
-   */
-  private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
-  throws IOException {
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.preLogRoll(oldPath, newPath);
-      }
-    }
-  }
-
-  /**
-   * Tell listeners about post log roll.
-   * @throws IOException
-   */
-  private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
-  throws IOException {
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.postLogRoll(oldPath, newPath);
-      }
-    }
+    return fsdos != null ? fsdos.getWrappedStream() : null;
   }
 
   /**
    * Run a sync after opening to set up the pipeline.
-   * @param nextWriter
-   * @param startTimeNanos
    */
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
@@ -665,115 +282,17 @@ public class FSHLog implements WAL {
     }
   }
 
-  @Override
-  public byte [][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
-    rollWriterLock.lock();
-    try {
-      // Return if nothing to flush.
-      if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null;
-      byte [][] regionsToFlush = null;
-      if (this.closed) {
-        LOG.debug("WAL closed. Skipping rolling of writer");
-        return regionsToFlush;
-      }
-      if (!closeBarrier.beginOp()) {
-        LOG.debug("WAL closing. Skipping rolling of writer");
-        return regionsToFlush;
-      }
-      TraceScope scope = Trace.startSpan("FSHLog.rollWriter");
-      try {
-        Path oldPath = getOldPath();
-        Path newPath = getNewPath();
-        // Any exception from here on is catastrophic, non-recoverable so we currently abort.
-        Writer nextWriter = this.createWriterInstance(newPath);
-        FSDataOutputStream nextHdfsOut = null;
-        if (nextWriter instanceof ProtobufLogWriter) {
-          nextHdfsOut = ((ProtobufLogWriter)nextWriter).getStream();
-          // If a ProtobufLogWriter, go ahead and try and sync to force setup of pipeline.
-          // If this fails, we just keep going.... it is an optimization, not the end of the world.
-          preemptiveSync((ProtobufLogWriter)nextWriter);
-        }
-        tellListenersAboutPreLogRoll(oldPath, newPath);
-        // NewPath could be equal to oldPath if replaceWriter fails.
-        newPath = replaceWriter(oldPath, newPath, nextWriter, nextHdfsOut);
-        tellListenersAboutPostLogRoll(oldPath, newPath);
-        // Can we delete any of the old log files?
-        if (getNumRolledLogFiles() > 0) {
-          cleanOldLogs();
-          regionsToFlush = findRegionsToForceFlush();
-        }
-      } finally {
-        closeBarrier.endOp();
-        assert scope == NullScope.INSTANCE || !scope.isDetached();
-        scope.close();
-      }
-      return regionsToFlush;
-    } finally {
-      rollWriterLock.unlock();
-    }
-  }
-
   /**
-   * This method allows subclasses to inject different writers without having to
-   * extend other methods like rollWriter().
-   *
+   * This method allows subclasses to inject different writers without having to extend other
+   * methods like rollWriter().
    * @return Writer instance
    */
   protected Writer createWriterInstance(final Path path) throws IOException {
-    return DefaultWALProvider.createWriter(conf, fs, path, false);
-  }
-
-  /**
-   * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
-   * @throws IOException
-   */
-  private void cleanOldLogs() throws IOException {
-    List<Path> logsToArchive = null;
-    // For each log file, look at its Map of regions to highest sequence id; if all sequence ids
-    // are older than what is currently in memory, the WAL can be GC'd.
-    for (Map.Entry<Path, Map<byte[], Long>> e : this.byWalRegionSequenceIds.entrySet()) {
-      Path log = e.getKey();
-      Map<byte[], Long> sequenceNums = e.getValue();
-      if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
-        if (logsToArchive == null) logsToArchive = new ArrayList<Path>();
-        logsToArchive.add(log);
-        if (LOG.isTraceEnabled()) LOG.trace("WAL file ready for archiving " + log);
-      }
-    }
-    if (logsToArchive != null) {
-      for (Path p : logsToArchive) {
-        this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
-        archiveLogFile(p);
-        this.byWalRegionSequenceIds.remove(p);
-      }
+    Writer writer = DefaultWALProvider.createWriter(conf, fs, path, false);
+    if (writer instanceof ProtobufLogWriter) {
+      preemptiveSync((ProtobufLogWriter) writer);
     }
-  }
-
-  /**
-   * If the number of un-archived WAL files is greater than maximum allowed, check the first
-   * (oldest) WAL file, and returns those regions which should be flushed so that it can
-   * be archived.
-   * @return regions (encodedRegionNames) to flush in order to archive oldest WAL file.
-   * @throws IOException
-   */
-  byte[][] findRegionsToForceFlush() throws IOException {
-    byte [][] regions = null;
-    int logCount = getNumRolledLogFiles();
-    if (logCount > this.maxLogs && logCount > 0) {
-      Map.Entry<Path, Map<byte[], Long>> firstWALEntry =
-        this.byWalRegionSequenceIds.firstEntry();
-      regions = this.sequenceIdAccounting.findLower(firstWALEntry.getValue());
-    }
-    if (regions != null) {
-      StringBuilder sb = new StringBuilder();
-      for (int i = 0; i < regions.length; i++) {
-        if (i > 0) sb.append(", ");
-        sb.append(Bytes.toStringBinary(regions[i]));
-      }
-      LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs +
-        "; forcing flush of " + regions.length + " regions(s): " + sb.toString());
-    }
-    return regions;
+    return writer;
   }
 
   /**
@@ -781,51 +300,39 @@ public class FSHLog implements WAL {
    * @see #beforeWaitOnSafePoint()
    */
   @VisibleForTesting
-  protected void afterCreatingZigZagLatch() {}
+  protected void afterCreatingZigZagLatch() {
+  }
 
   /**
    * @see #afterCreatingZigZagLatch()
    */
   @VisibleForTesting
-  protected void beforeWaitOnSafePoint() {};
+  protected void beforeWaitOnSafePoint() {
+  };
 
-  /**
-   * Cleans up current writer closing it and then puts in place the passed in
-   * <code>nextWriter</code>.
-   *
-   * In the case of creating a new WAL, oldPath will be null.
-   *
-   * In the case of rolling over from one file to the next, none of the params will be null.
-   *
-   * In the case of closing out this FSHLog with no further use newPath, nextWriter, and
-   * nextHdfsOut will be null.
-   *
-   * @param oldPath may be null
-   * @param newPath may be null
-   * @param nextWriter may be null
-   * @param nextHdfsOut may be null
-   * @return the passed in <code>newPath</code>
-   * @throws IOException if there is a problem flushing or closing the underlying FS
-   */
-  Path replaceWriter(final Path oldPath, final Path newPath, Writer nextWriter,
-      final FSDataOutputStream nextHdfsOut)
-  throws IOException {
-    // Ask the ring buffer writer to pause at a safe point.  Once we do this, the writer
+  @Override
+  protected void doAppend(Writer writer, FSWALEntry entry) throws IOException {
+    writer.append(entry);
+  }
+
+  @Override
+  protected long doReplaceWriter(Path oldPath, Path newPath, Writer nextWriter) throws IOException {
+    // Ask the ring buffer writer to pause at a safe point. Once we do this, the writer
     // thread will eventually pause. An error hereafter needs to release the writer thread
-    // regardless -- hence the finally block below.  Note, this method is called from the FSHLog
+    // regardless -- hence the finally block below. Note, this method is called from the FSHLog
     // constructor BEFORE the ring buffer is set running so it is null on first time through
     // here; allow for that.
     SyncFuture syncFuture = null;
-    SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null)?
-      null: this.ringBufferEventHandler.attainSafePoint();
+    SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null) ? null
+        : this.ringBufferEventHandler.attainSafePoint();
     afterCreatingZigZagLatch();
-    TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
+    long oldFileLen = 0L;
     try {
-      // Wait on the safe point to be achieved.  Send in a sync in case nothing has hit the
+      // Wait on the safe point to be achieved. Send in a sync in case nothing has hit the
       // ring buffer between the above notification of writer that we want it to go to
-      // 'safe point' and then here where we are waiting on it to attain safe point.  Use
+      // 'safe point' and then here where we are waiting on it to attain safe point. Use
       // 'sendSync' instead of 'sync' because we do not want this thread to block waiting on it
-      // to come back.  Cleanup this syncFuture down below after we are ready to run again.
+      // to come back. Cleanup this syncFuture down below after we are ready to run again.
       try {
         if (zigzagLatch != null) {
           Trace.addTimelineAnnotation("awaiting safepoint");
@@ -833,44 +340,37 @@ public class FSHLog implements WAL {
         }
       } catch (FailedSyncBeforeLogCloseException e) {
         // If unflushed/unsynced entries on close, it is reason to abort.
-        if (isUnflushedEntries()) throw e;
-        LOG.warn("Failed sync-before-close but no outstanding appends; closing WAL: " +
-          e.getMessage());
+        if (isUnflushedEntries()) {
+          throw e;
+        }
+        LOG.warn(
+          "Failed sync-before-close but no outstanding appends; closing WAL" + e.getMessage());
       }
-
-      // It is at the safe point.  Swap out writer from under the blocked writer thread.
-      // TODO: This is close is inline with critical section.  Should happen in background?
-      try {
-        if (this.writer != null) {
+      // It is at the safe point. Swap out writer from under the blocked writer thread.
+      // TODO: This is close is inline with critical section. Should happen in background?
+      if (this.writer != null) {
+        oldFileLen = this.writer.getLength();
+        try {
           Trace.addTimelineAnnotation("closing writer");
           this.writer.close();
           Trace.addTimelineAnnotation("writer closed");
-        }
-        this.closeErrorCount.set(0);
-      } catch (IOException ioe) {
-        int errors = closeErrorCount.incrementAndGet();
-        if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) {
-          LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" +
-            ioe.getMessage() + "\", errors=" + errors +
-            "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK");
-        } else {
-          throw ioe;
+          this.closeErrorCount.set(0);
+        } catch (IOException ioe) {
+          int errors = closeErrorCount.incrementAndGet();
+          if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) {
+            LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" + ioe.getMessage()
+                + "\", errors=" + errors
+                + "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK");
+          } else {
+            throw ioe;
+          }
         }
       }
       this.writer = nextWriter;
-      this.hdfs_out = nextHdfsOut;
-      int oldNumEntries = this.numEntries.get();
-      this.numEntries.set(0);
-      final String newPathString = (null == newPath ? null : FSUtils.getPath(newPath));
-      if (oldPath != null) {
-        this.byWalRegionSequenceIds.put(oldPath, this.sequenceIdAccounting.resetHighest());
-        long oldFileLen = this.fs.getFileStatus(oldPath).getLen();
-        this.totalLogSize.addAndGet(oldFileLen);
-        LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries +
-          ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " +
-          newPathString);
+      if (nextWriter != null && nextWriter instanceof ProtobufLogWriter) {
+        this.hdfs_out = ((ProtobufLogWriter) nextWriter).getStream();
       } else {
-        LOG.info("New WAL " + newPathString);
+        this.hdfs_out = null;
       }
     } catch (InterruptedException ie) {
       // Perpetuate the interrupt
@@ -880,223 +380,84 @@ public class FSHLog implements WAL {
       LOG.error("Failed close of WAL writer " + oldPath + ", unflushedEntries=" + count, e);
       throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e);
     } finally {
-      try {
-        // Let the writer thread go regardless, whether error or not.
-        if (zigzagLatch != null) {
-          zigzagLatch.releaseSafePoint();
-          // syncFuture will be null if we failed our wait on safe point above. Otherwise, if
-          // latch was obtained successfully, the sync we threw in either trigger the latch or it
-          // got stamped with an exception because the WAL was damaged and we could not sync. Now
-          // the write pipeline has been opened up again by releasing the safe point, process the
-          // syncFuture we got above. This is probably a noop but it may be stale exception from
-          // when old WAL was in place. Catch it if so.
-          if (syncFuture != null) {
-            try {
-              blockOnSync(syncFuture);
-            } catch (IOException ioe) {
-              if (LOG.isTraceEnabled()) LOG.trace("Stale sync exception", ioe);
+      // Let the writer thread go regardless, whether error or not.
+      if (zigzagLatch != null) {
+        zigzagLatch.releaseSafePoint();
+        // syncFuture will be null if we failed our wait on safe point above. Otherwise, if
+        // latch was obtained successfully, the sync we threw in either trigger the latch or it
+        // got stamped with an exception because the WAL was damaged and we could not sync. Now
+        // the write pipeline has been opened up again by releasing the safe point, process the
+        // syncFuture we got above. This is probably a noop but it may be stale exception from
+        // when old WAL was in place. Catch it if so.
+        if (syncFuture != null) {
+          try {
+            blockOnSync(syncFuture);
+          } catch (IOException ioe) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Stale sync exception", ioe);
             }
           }
         }
-      } finally {
-        scope.close();
       }
     }
-    return newPath;
-  }
-
-  long getUnflushedEntriesCount() {
-    long highestSynced = this.highestSyncedSequence.get();
-    return highestSynced > this.highestUnsyncedSequence?
-      0: this.highestUnsyncedSequence - highestSynced;
-  }
-
-  boolean isUnflushedEntries() {
-    return getUnflushedEntriesCount() > 0;
+    return oldFileLen;
   }
 
-  /*
-   * only public so WALSplitter can use.
-   * @return archived location of a WAL file with the given path p
-   */
-  public static Path getWALArchivePath(Path archiveDir, Path p) {
-    return new Path(archiveDir, p.getName());
-  }
-
-  private void archiveLogFile(final Path p) throws IOException {
-    Path newPath = getWALArchivePath(this.fullPathArchiveDir, p);
-    // Tell our listeners that a log is going to be archived.
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.preLogArchive(p, newPath);
-      }
-    }
-    LOG.info("Archiving " + p + " to " + newPath);
-    if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
-      throw new IOException("Unable to rename " + p + " to " + newPath);
-    }
-    // Tell our listeners that a log has been archived.
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i : this.listeners) {
-        i.postLogArchive(p, newPath);
+  @Override
+  protected void doShutdown() throws IOException {
+    // Shutdown the disruptor. Will stop after all entries have been processed. Make sure we
+    // have stopped incoming appends before calling this else it will not shutdown. We are
+    // conservative below waiting a long time and if not elapsed, then halting.
+    if (this.disruptor != null) {
+      long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000);
+      try {
+        this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
+      } catch (TimeoutException e) {
+        LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt "
+            + "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)");
+        this.disruptor.halt();
+        this.disruptor.shutdown();
       }
     }
-  }
-
-  /**
-   * This is a convenience method that computes a new filename with a given
-   * file-number.
-   * @param filenum to use
-   * @return Path
-   */
-  protected Path computeFilename(final long filenum) {
-    if (filenum < 0) {
-      throw new RuntimeException("WAL file number can't be < 0");
+    // With disruptor down, this is safe to let go.
+    if (this.appendExecutor != null) {
+      this.appendExecutor.shutdown();
     }
-    String child = logFilePrefix + WAL_FILE_NAME_DELIMITER + filenum + logFileSuffix;
-    return new Path(fullPathLogDir, child);
-  }
-
-  /**
-   * This is a convenience method that computes a new filename with a given
-   * using the current WAL file-number
-   * @return Path
-   */
-  public Path getCurrentFileName() {
-    return computeFilename(this.filenum.get());
-  }
-
-  @Override
-  public String toString() {
-    return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")";
-  }
 
-/**
- * A log file has a creation timestamp (in ms) in its file name ({@link #filenum}.
- * This helper method returns the creation timestamp from a given log file.
- * It extracts the timestamp assuming the filename is created with the
- * {@link #computeFilename(long filenum)} method.
- * @param fileName
- * @return timestamp, as in the log file name.
- */
-  protected long getFileNumFromFileName(Path fileName) {
-    if (fileName == null) throw new IllegalArgumentException("file name can't be null");
-    if (!ourFiles.accept(fileName)) {
-      throw new IllegalArgumentException("The log file " + fileName +
-          " doesn't belong to this WAL. (" + toString() + ")");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Closing WAL writer in " + FSUtils.getPath(walDir));
     }
-    final String fileNameString = fileName.toString();
-    String chompedPath = fileNameString.substring(prefixPathStr.length(),
-        (fileNameString.length() - logFileSuffix.length()));
-    return Long.parseLong(chompedPath);
-  }
-
-  @Override
-  public void close() throws IOException {
-    shutdown();
-    final FileStatus[] files = getFiles();
-    if (null != files && 0 != files.length) {
-      for (FileStatus file : files) {
-        Path p = getWALArchivePath(this.fullPathArchiveDir, file.getPath());
-        // Tell our listeners that a log is going to be archived.
-        if (!this.listeners.isEmpty()) {
-          for (WALActionsListener i : this.listeners) {
-            i.preLogArchive(file.getPath(), p);
-          }
-        }
-
-        if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
-          throw new IOException("Unable to rename " + file.getPath() + " to " + p);
-        }
-        // Tell our listeners that a log was archived.
-        if (!this.listeners.isEmpty()) {
-          for (WALActionsListener i : this.listeners) {
-            i.postLogArchive(file.getPath(), p);
-          }
-        }
-      }
-      LOG.debug("Moved " + files.length + " WAL file(s) to " +
-        FSUtils.getPath(this.fullPathArchiveDir));
+    if (this.writer != null) {
+      this.writer.close();
+      this.writer = null;
     }
-    LOG.info("Closed WAL: " + toString());
   }
 
   @Override
-  public void shutdown() throws IOException {
-    if (shutdown.compareAndSet(false, true)) {
-      try {
-        // Prevent all further flushing and rolling.
-        closeBarrier.stopAndDrainOps();
-      } catch (InterruptedException e) {
-        LOG.error("Exception while waiting for cache flushes and log rolls", e);
-        Thread.currentThread().interrupt();
-      }
-
-      // Shutdown the disruptor.  Will stop after all entries have been processed.  Make sure we
-      // have stopped incoming appends before calling this else it will not shutdown.  We are
-      // conservative below waiting a long time and if not elapsed, then halting.
-      if (this.disruptor != null) {
-        long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000);
-        try {
-          this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
-        } catch (TimeoutException e) {
-          LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " +
-            "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)");
-          this.disruptor.halt();
-          this.disruptor.shutdown();
-        }
-      }
-      // With disruptor down, this is safe to let go.
-      if (this.appendExecutor !=  null) this.appendExecutor.shutdown();
-
-      // Tell our listeners that the log is closing
-      if (!this.listeners.isEmpty()) {
-        for (WALActionsListener i : this.listeners) {
-          i.logCloseRequested();
-        }
-      }
-      this.closed = true;
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Closing WAL writer in " + FSUtils.getPath(fullPathLogDir));
-      }
-      if (this.writer != null) {
-        this.writer.close();
-        this.writer = null;
-      }
-    }
+  public String toString() {
+    return "FSHLog " + walFilePrefix + ":" + walFileSuffix + "(num " + filenum + ")";
   }
 
-  /**
-   * NOTE: This append, at a time that is usually after this call returns, starts an
-   * mvcc transaction by calling 'begin' wherein which we assign this update a sequenceid. At
-   * assignment time, we stamp all the passed in Cells inside WALEdit with their sequenceId.
-   * You must 'complete' the transaction this mvcc transaction by calling
-   * MultiVersionConcurrencyControl#complete(...) or a variant otherwise mvcc will get stuck. Do it
-   * in the finally of a try/finally
-   * block within which this append lives and any subsequent operations like sync or
-   * update of memstore, etc. Get the WriteEntry to pass mvcc out of the passed in WALKey
-   * <code>walKey</code> parameter. Be warned that the WriteEntry is not immediately available
-   * on return from this method. It WILL be available subsequent to a sync of this append;
-   * otherwise, you will just have to wait on the WriteEntry to get filled in.
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
-      justification="Will never be null")
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH_EXCEPTION",
+      justification = "Will never be null")
   @Override
   public long append(final HRegionInfo hri,
       final WALKey key, final WALEdit edits, final boolean inMemstore) throws IOException {
-    if (this.closed) throw new IOException("Cannot append; log is closed");
-    // Make a trace scope for the append.  It is closed on other side of the ring buffer by the
-    // single consuming thread.  Don't have to worry about it.
+    if (this.closed) {
+      throw new IOException("Cannot append; log is closed");
+    }
+    // Make a trace scope for the append. It is closed on other side of the ring buffer by the
+    // single consuming thread. Don't have to worry about it.
     TraceScope scope = Trace.startSpan("FSHLog.append");
 
-    // This is crazy how much it takes to make an edit.  Do we need all this stuff!!!!????  We need
+    // This is crazy how much it takes to make an edit. Do we need all this stuff!!!!???? We need
     // all this to make a key and then below to append the edit, we need to carry htd, info,
     // etc. all over the ring buffer.
     FSWALEntry entry = null;
     long sequence = this.disruptor.getRingBuffer().next();
     try {
       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
-      // Construction of FSWALEntry sets a latch.  The latch is thrown just after we stamp the
+      // Construction of FSWALEntry sets a latch. The latch is thrown just after we stamp the
       // edit with its edit/sequence id.
       // TODO: reuse FSWALEntry as we do SyncFuture rather create per append.
       entry = new FSWALEntry(sequence, key, edits, hri, inMemstore);
@@ -1108,17 +469,18 @@ public class FSHLog implements WAL {
   }
 
   /**
-   * Thread to runs the hdfs sync call. This call takes a while to complete.  This is the longest
-   * pole adding edits to the WAL and this must complete to be sure all edits persisted.  We run
-   * multiple threads sync'ng rather than one that just syncs in series so we have better
-   * latencies; otherwise, an edit that arrived just after a sync started, might have to wait
-   * almost the length of two sync invocations before it is marked done.
-   * <p>When the sync completes, it marks all the passed in futures done.  On the other end of the
-   * sync future is a blocked thread, usually a regionserver Handler.  There may be more than one
-   * future passed in the case where a few threads arrive at about the same time and all invoke
-   * 'sync'.  In this case we'll batch up the invocations and run one filesystem sync only for a
-   * batch of Handler sync invocations.  Do not confuse these Handler SyncFutures with the futures
-   * an ExecutorService returns when you call submit. We have no use for these in this model. These
+   * Thread to runs the hdfs sync call. This call takes a while to complete. This is the longest
+   * pole adding edits to the WAL and this must complete to be sure all edits persisted. We run
+   * multiple threads sync'ng rather than one that just syncs in series so we have better latencies;
+   * otherwise, an edit that arrived just after a sync started, might have to wait almost the length
+   * of two sync invocations before it is marked done.
+   * <p>
+   * When the sync completes, it marks all the passed in futures done. On the other end of the sync
+   * future is a blocked thread, usually a regionserver Handler. There may be more than one future
+   * passed in the case where a few threads arrive at about the same time and all invoke 'sync'. In
+   * this case we'll batch up the invocations and run one filesystem sync only for a batch of
+   * Handler sync invocations. Do not confuse these Handler SyncFutures with the futures an
+   * ExecutorService returns when you call submit. We have no use for these in this model. These
    * SyncFutures are 'artificial', something to hold the Handler until the filesystem sync
    * completes.
    */
@@ -1130,12 +492,13 @@ public class FSHLog implements WAL {
     /**
      * UPDATE!
      * @param syncs the batch of calls to sync that arrived as this thread was starting; when done,
-     * we will put the result of the actual hdfs sync call as the result.
-     * @param sequence The sequence number on the ring buffer when this thread was set running.
-     * If this actual writer sync completes then all appends up this point have been
-     * flushed/synced/pushed to datanodes.  If we fail, then the passed in <code>syncs</code>
-     * futures will return the exception to their clients; some of the edits may have made it out
-     * to data nodes but we will report all that were part of this session as failed.
+     *          we will put the result of the actual hdfs sync call as the result.
+     * @param sequence The sequence number on the ring buffer when this thread was set running. If
+     *          this actual writer sync completes then all appends up this point have been
+     *          flushed/synced/pushed to datanodes. If we fail, then the passed in
+     *          <code>syncs</code> futures will return the exception to their clients; some of the
+     *          edits may have made it out to data nodes but we will report all that were part of
+     *          this session as failed.
      */
     SyncRunner(final String name, final int maxHandlersCount) {
       super(name);
@@ -1145,17 +508,17 @@ public class FSHLog implements WAL {
       //
       // We could let the capacity be 'open' but bound it so we get alerted in pathological case
       // where we cannot sync and we have a bunch of threads all backed up waiting on their syncs
-      // to come in.  LinkedBlockingQueue actually shrinks when you remove elements so Q should
-      // stay neat and tidy in usual case.  Let the max size be three times the maximum handlers.
+      // to come in. LinkedBlockingQueue actually shrinks when you remove elements so Q should
+      // stay neat and tidy in usual case. Let the max size be three times the maximum handlers.
       // The passed in maxHandlerCount is the user-level handlers which is what we put up most of
       // but HBase has other handlers running too -- opening region handlers which want to write
-      // the meta table when succesful (i.e. sync), closing handlers -- etc.  These are usually
+      // the meta table when succesful (i.e. sync), closing handlers -- etc. These are usually
       // much fewer in number than the user-space handlers so Q-size should be user handlers plus
-      // some space for these other handlers.  Lets multiply by 3 for good-measure.
+      // some space for these other handlers. Lets multiply by 3 for good-measure.
       this.syncFutures = new LinkedBlockingQueue<SyncFuture>(maxHandlersCount * 3);
     }
 
-    void offer(final long sequence, final SyncFuture [] syncFutures, final int syncFutureCount) {
+    void offer(final long sequence, final SyncFuture[] syncFutures, final int syncFutureCount) {
       // Set sequence first because the add to the queue will wake the thread if sleeping.
       this.sequence = sequence;
       for (int i = 0; i < syncFutureCount; ++i) {
@@ -1165,28 +528,28 @@ public class FSHLog implements WAL {
 
     /**
      * Release the passed <code>syncFuture</code>
-     * @param syncFuture
-     * @param currentSequence
-     * @param t
      * @return Returns 1.
      */
     private int releaseSyncFuture(final SyncFuture syncFuture, final long currentSequence,
         final Throwable t) {
-      if (!syncFuture.done(currentSequence, t)) throw new IllegalStateException();
+      if (!syncFuture.done(currentSequence, t)) {
+        throw new IllegalStateException();
+      }
       // This function releases one sync future only.
       return 1;
     }
 
     /**
      * Release all SyncFutures whose sequence is <= <code>currentSequence</code>.
-     * @param currentSequence
      * @param t May be non-null if we are processing SyncFutures because an exception was thrown.
      * @return Count of SyncFutures we let go.
      */
     private int releaseSyncFutures(final long currentSequence, final Throwable t) {
       int syncCount = 0;
       for (SyncFuture syncFuture; (syncFuture = this.syncFutures.peek()) != null;) {
-        if (syncFuture.getRingBufferSequence() > currentSequence) break;
+        if (syncFuture.getTxid() > currentSequence) {
+          break;
+        }
         releaseSyncFuture(syncFuture, currentSequence, t);
         if (!this.syncFutures.remove(syncFuture)) {
           throw new IllegalStateException(syncFuture.toString());
@@ -1204,14 +567,14 @@ public class FSHLog implements WAL {
       long currentHighestSyncedSequence;
       // Set the highestSyncedSequence IFF our current sequence id is the 'highest'.
       do {
-        currentHighestSyncedSequence = highestSyncedSequence.get();
+        currentHighestSyncedSequence = highestSyncedTxid.get();
         if (currentHighestSyncedSequence >= sequence) {
           // Set the sync number to current highwater mark; might be able to let go more
           // queued sync futures
           sequence = currentHighestSyncedSequence;
           break;
         }
-      } while (!highestSyncedSequence.compareAndSet(currentHighestSyncedSequence, sequence));
+      } while (!highestSyncedTxid.compareAndSet(currentHighestSyncedSequence, sequence));
       return sequence;
     }
 
@@ -1225,21 +588,21 @@ public class FSHLog implements WAL {
             // We have to process what we 'take' from the queue
             takeSyncFuture = this.syncFutures.take();
             currentSequence = this.sequence;
-            long syncFutureSequence = takeSyncFuture.getRingBufferSequence();
+            long syncFutureSequence = takeSyncFuture.getTxid();
             if (syncFutureSequence > currentSequence) {
-              throw new IllegalStateException("currentSequence=" + syncFutureSequence +
-                ", syncFutureSequence=" + syncFutureSequence);
+              throw new IllegalStateException("currentSequence=" + syncFutureSequence
+                  + ", syncFutureSequence=" + syncFutureSequence);
             }
             // See if we can process any syncfutures BEFORE we go sync.
-            long currentHighestSyncedSequence = highestSyncedSequence.get();
+            long currentHighestSyncedSequence = highestSyncedTxid.get();
             if (currentSequence < currentHighestSyncedSequence) {
               syncCount += releaseSyncFuture(takeSyncFuture, currentHighestSyncedSequence, null);
-              // Done with the 'take'.  Go around again and do a new 'take'.
+              // Done with the 'take'. Go around again and do a new 'take'.
               continue;
             }
             break;
           }
-          // I got something.  Lets run.  Save off current sequence number in case it changes
+          // I got something. Lets run. Save off current sequence number in case it changes
           // while we run.
           TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan());
           long start = System.nanoTime();
@@ -1262,8 +625,11 @@ public class FSHLog implements WAL {
             syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, lastException);
             // Can we release other syncs?
             syncCount += releaseSyncFutures(currentSequence, lastException);
-            if (lastException != null) requestLogRoll();
-            else checkLogRoll();
+            if (lastException != null) {
+              requestLogRoll();
+            } else {
+              checkLogRoll();
+            }
           }
           postSync(System.nanoTime() - start, syncCount);
         } catch (InterruptedException e) {
@@ -1281,7 +647,9 @@ public class FSHLog implements WAL {
    */
   void checkLogRoll() {
     // Will return immediately if we are in the middle of a WAL log roll currently.
-    if (!rollWriterLock.tryLock()) return;
+    if (!rollWriterLock.tryLock()) {
+      return;
+    }
     boolean lowReplication;
     try {
       lowReplication = checkLowReplication();
@@ -1297,7 +665,7 @@ public class FSHLog implements WAL {
     }
   }
 
-  /*
+  /**
    * @return true if number of replicas for the WAL is lower than threshold
    */
   private boolean checkLowReplication() {
@@ -1309,11 +677,10 @@ public class FSHLog implements WAL {
       if (numCurrentReplicas != 0 && numCurrentReplicas < this.minTolerableReplication) {
         if (this.lowReplicationRollEnabled) {
           if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) {
-            LOG.warn("HDFS pipeline error detected. " + "Found "
-                + numCurrentReplicas + " replicas but expecting no less than "
-                + this.minTolerableReplication + " replicas. "
-                + " Requesting close of WAL. current pipeline: "
-                + Arrays.toString(getPipeLine()));
+            LOG.warn("HDFS pipeline error detected. " + "Found " + numCurrentReplicas
+                + " replicas but expecting no less than " + this.minTolerableReplication
+                + " replicas. " + " Requesting close of WAL. current pipeline: "
+                + Arrays.toString(getPipeline()));
             logRollNeeded = true;
             // If rollWriter is requested, increase consecutiveLogRolls. Once it
             // is larger than lowReplicationRollLimit, disable the
@@ -1341,8 +708,7 @@ public class FSHLog implements WAL {
         }
       }
     } catch (Exception e) {
-      LOG.warn("DFSOutputStream.getNumCurrentReplicas failed because of " + e +
-        ", continuing...");
+      LOG.warn("DFSOutputStream.getNumCurrentReplicas failed because of " + e + ", continuing...");
     }
     return logRollNeeded;
   }
@@ -1353,6 +719,7 @@ public class FSHLog implements WAL {
 
   private SyncFuture publishSyncOnRingBuffer(Span span) {
     long sequence = this.disruptor.getRingBuffer().next();
+    // here we use ring buffer sequence as transaction id
     SyncFuture syncFuture = getSyncFuture(sequence, span);
     try {
       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
@@ -1368,81 +735,17 @@ public class FSHLog implements WAL {
     return blockOnSync(publishSyncOnRingBuffer(span));
   }
 
-  private Span blockOnSync(final SyncFuture syncFuture) throws IOException {
-    // Now we have published the ringbuffer, halt the current thread until we get an answer back.
-    try {
-      syncFuture.get();
-      return syncFuture.getSpan();
-    } catch (InterruptedException ie) {
-      LOG.warn("Interrupted", ie);
-      throw convertInterruptedExceptionToIOException(ie);
-    } catch (ExecutionException e) {
-      throw ensureIOException(e.getCause());
-    }
-  }
-
-  private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
-    Thread.currentThread().interrupt();
-    IOException ioe = new InterruptedIOException();
-    ioe.initCause(ie);
-    return ioe;
-  }
-
-  private SyncFuture getSyncFuture(final long sequence, Span span) {
-    SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
-    if (syncFuture == null) {
-      syncFuture = new SyncFuture();
-      this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
-    }
-    return syncFuture.reset(sequence, span);
-  }
-
-  private void postSync(final long timeInNanos, final int handlerSyncs) {
-    if (timeInNanos > this.slowSyncNs) {
-      String msg =
-          new StringBuilder().append("Slow sync cost: ")
-              .append(timeInNanos / 1000000).append(" ms, current pipeline: ")
-              .append(Arrays.toString(getPipeLine())).toString();
-      Trace.addTimelineAnnotation(msg);
-      LOG.info(msg);
-    }
-    if (!listeners.isEmpty()) {
-      for (WALActionsListener listener : listeners) {
-        listener.postSync(timeInNanos, handlerSyncs);
-      }
-    }
-  }
-
-  private long postAppend(final Entry e, final long elapsedTime) {
-    long len = 0;
-    if (!listeners.isEmpty()) {
-      for (Cell cell : e.getEdit().getCells()) {
-        len += CellUtil.estimatedSerializedSizeOf(cell);
-      }
-      for (WALActionsListener listener : listeners) {
-        listener.postAppend(len, elapsedTime);
-      }
-    }
-    return len;
-  }
-
-
   /**
-   * This method gets the datanode replication count for the current WAL.
-   *
-   * If the pipeline isn't started yet or is empty, you will get the default
-   * replication factor.  Therefore, if this function returns 0, it means you
-   * are not properly running with the HDFS-826 patch.
-   * @throws InvocationTargetException
-   * @throws IllegalAccessException
-   * @throws IllegalArgumentException
-   *
-   * @throws Exception
+   * {@inheritDoc}
+   * <p>
+   * If the pipeline isn't started yet or is empty, you will get the default replication factor.
+   * Therefore, if this function returns 0, it means you are not properly running with the HDFS-826
+   * patch.
    */
   @VisibleForTesting
   int getLogReplication() {
     try {
-      //in standalone mode, it will return 0
+      // in standalone mode, it will return 0
       if (this.hdfs_out instanceof HdfsDataOutputStream) {
         return ((HdfsDataOutputStream) this.hdfs_out).getCurrentBlockReplication();
       }
@@ -1465,7 +768,7 @@ public class FSHLog implements WAL {
 
   @Override
   public void sync(long txid) throws IOException {
-    if (this.highestSyncedSequence.get() >= txid){
+    if (this.highestSyncedTxid.get() >= txid) {
       // Already sync'd.
       return;
     }
@@ -1478,70 +781,20 @@ public class FSHLog implements WAL {
     }
   }
 
-  // public only until class moves to o.a.h.h.wal
-  public void requestLogRoll() {
-    requestLogRoll(false);
-  }
-
-  private void requestLogRoll(boolean tooFewReplicas) {
-    if (!this.listeners.isEmpty()) {
-      for (WALActionsListener i: this.listeners) {
-        i.logRollRequested(tooFewReplicas);
-      }
-    }
-  }
-
-  // public only until class moves to o.a.h.h.wal
-  /** @return the number of rolled log files */
-  public int getNumRolledLogFiles() {
-    return byWalRegionSequenceIds.size();
-  }
-
-  // public only until class moves to o.a.h.h.wal
-  /** @return the number of log files in use */
-  public int getNumLogFiles() {
-    // +1 for current use log
-    return getNumRolledLogFiles() + 1;
-  }
-
-  // public only until class moves to o.a.h.h.wal
-  /** @return the size of log files in use */
-  public long getLogFileSize() {
-    return this.totalLogSize.get();
-  }
-
-  @Override
-  public Long startCacheFlush(final byte[] encodedRegionName, Set<byte[]> families) {
-    if (!closeBarrier.beginOp()) {
-      LOG.info("Flush not started for " + Bytes.toString(encodedRegionName) + "; server closing.");
-      return null;
-    }
-    return this.sequenceIdAccounting.startCacheFlush(encodedRegionName, families);
-  }
-
-  @Override
-  public void completeCacheFlush(final byte [] encodedRegionName) {
-    this.sequenceIdAccounting.completeCacheFlush(encodedRegionName);
-    closeBarrier.endOp();
-  }
-
   @Override
-  public void abortCacheFlush(byte[] encodedRegionName) {
-    this.sequenceIdAccounting.abortCacheFlush(encodedRegionName);
-    closeBarrier.endOp();
+  public void logRollerExited() {
   }
 
   @VisibleForTesting
   boolean isLowReplicationRollEnabled() {
-      return lowReplicationRollEnabled;
+    return lowReplicationRollEnabled;
   }
 
-  public static final long FIXED_OVERHEAD = ClassSize.align(
-    ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
-    ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
+  public static final long FIXED_OVERHEAD = ClassSize
+      .align(ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + ClassSize.ATOMIC_INTEGER
+          + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
 
-  private static void split(final Configuration conf, final Path p)
-  throws IOException {
+  private static void split(final Configuration conf, final Path p) throws IOException {
     FileSystem fs = FileSystem.get(conf);
     if (!fs.exists(p)) {
       throw new FileNotFoundException(p.toString());
@@ -1555,51 +808,30 @@ public class FSHLog implements WAL {
     WALSplitter.split(baseDir, p, archiveDir, fs, conf, WALFactory.getInstance(conf));
   }
 
-
-  @Override
-  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
-    // Used by tests. Deprecated as too subtle for general usage.
-    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName);
-  }
-
-  @Override
-  public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) {
-    // This method is used by tests and for figuring if we should flush or not because our
-    // sequenceids are too old. It is also used reporting the master our oldest sequenceid for use
-    // figuring what edits can be skipped during log recovery. getEarliestMemStoreSequenceId
-    // from this.sequenceIdAccounting is looking first in flushingOldestStoreSequenceIds, the
-    // currently flushing sequence ids, and if anything found there, it is returning these. This is
-    // the right thing to do for the reporting oldest sequenceids to master; we won't skip edits if
-    // we crash during the flush. For figuring what to flush, we might get requeued if our sequence
-    // id is old even though we are currently flushing. This may mean we do too much flushing.
-    return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName, familyName);
-  }
-
   /**
-   * This class is used coordinating two threads holding one thread at a
-   * 'safe point' while the orchestrating thread does some work that requires the first thread
-   * paused: e.g. holding the WAL writer while its WAL is swapped out from under it by another
-   * thread.
-   *
-   * <p>Thread A signals Thread B to hold when it gets to a 'safe point'.  Thread A wait until
-   * Thread B gets there. When the 'safe point' has been attained, Thread B signals Thread A.
-   * Thread B then holds at the 'safe point'.  Thread A on notification that Thread B is paused,
-   * goes ahead and does the work it needs to do while Thread B is holding.  When Thread A is done,
-   * it flags B and then Thread A and Thread B continue along on their merry way.  Pause and
-   * signalling 'zigzags' between the two participating threads.  We use two latches -- one the
-   * inverse of the other -- pausing and signaling when states are achieved.
-   *
-   * <p>To start up the drama, Thread A creates an instance of this class each time it would do
-   * this zigzag dance and passes it to Thread B (these classes use Latches so it is one shot
-   * only). Thread B notices the new instance (via reading a volatile reference or how ever) and it
-   * starts to work toward the 'safe point'.  Thread A calls {@link #waitSafePoint()} when it
-   * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in
-   * {@link #waitSafePoint()} until Thread B reaches the 'safe point'.  Once there, Thread B
-   * frees Thread A by calling {@link #safePointAttained()}.  Thread A now knows Thread B
-   * is at the 'safe point' and that it is holding there (When Thread B calls
-   * {@link #safePointAttained()} it blocks here until Thread A calls {@link #releaseSafePoint()}).
-   * Thread A proceeds to do what it needs to do while Thread B is paused.  When finished,
-   * it lets Thread B lose by calling {@link #releaseSafePoint()} and away go both Threads again.
+   * This class is used coordinating two threads holding one thread at a 'safe point' while the
+   * orchestrating thread does some work that requires the first thread paused: e.g. holding the WAL
+   * writer while its WAL is swapped out from under it by another thread.
+   * <p>
+   * Thread A signals Thread B to hold when it gets to a 'safe point'. Thread A wait until Thread B
+   * gets there. When the 'safe point' has been attained, Thread B signals Thread A. Thread B then
+   * holds at the 'safe point'. Thread A on notification that Thread B is paused, goes ahead and
+   * does the work it needs to do while Thread B is holding. When Thread A is done, it flags B and
+   * then Thread A and Thread B continue along on their merry way. Pause and signalling 'zigzags'
+   * between the two participating threads. We use two latches -- one the inverse of the other --
+   * pausing and signaling when states are achieved.
+   * <p>
+   * To start up the drama, Thread A creates an instance of this class each time it would do this
+   * zigzag dance and passes it to Thread B (these classes use Latches so it is one shot only).
+   * Thread B notices the new instance (via reading a volatile reference or how ever) and it starts
+   * to work toward the 'safe point'. Thread A calls {@link #waitSafePoint()} when it cannot proceed
+   * until the Thread B 'safe point' is attained. Thread A will be held inside in
+   * {@link #waitSafePoint()} until Thread B reaches the 'safe point'. Once there, Thread B frees
+   * Thread A by calling {@link #safePointAttained()}. Thread A now knows Thread B is at the 'safe
+   * point' and that it is holding there (When Thread B calls {@link #safePointAttained()} it blocks
+   * here until Thread A calls {@link #releaseSafePoint()}). Thread A proceeds to do what it needs
+   * to do while Thread B is paused. When finished, it lets Thread B lose by calling
+   * {@link #releaseSafePoint()} and away go both Threads again.
    */
   static class SafePointZigZagLatch {
     /**
@@ -1607,24 +839,23 @@ public class FSHLog implements WAL {
      */
     private volatile CountDownLatch safePointAttainedLatch = new CountDownLatch(1);
     /**
-     * Latch to wait on.  Will be released when we can proceed.
+     * Latch to wait on. Will be released when we can proceed.
      */
     private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1);
 
     /**
-     * For Thread A to call when it is ready to wait on the 'safe point' to be attained.
-     * Thread A will be held in here until Thread B calls {@link #safePointAttained()}
-     * @param syncFuture We need this as barometer on outstanding syncs.  If it comes home with
-     * an exception, then something is up w/ our syncing.
-     * @throws InterruptedException
-     * @throws ExecutionException
+     * For Thread A to call when it is ready to wait on the 'safe point' to be attained. Thread A
+     * will be held in here until Thread B calls {@link #safePointAttained()}
+     * @param syncFuture We need this as barometer on outstanding syncs. If it comes home with an
+     *          exception, then something is up w/ our syncing.
      * @return The passed <code>syncFuture</code>
-     * @throws FailedSyncBeforeLogCloseException
      */
-    SyncFuture waitSafePoint(final SyncFuture syncFuture)
-    throws InterruptedException, FailedSyncBeforeLogCloseException {
+    SyncFuture waitSafePoint(final SyncFuture syncFuture) throws InterruptedException,
+        FailedSyncBeforeLogCloseException {
       while (true) {
-        if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) break;
+        if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) {
+          break;
+        }
         if (syncFuture.isThrowable()) {
           throw new FailedSyncBeforeLogCloseException(syncFuture.getThrowable());
         }
@@ -1633,10 +864,9 @@ public class FSHLog implements WAL {
     }
 
     /**
-     * Called by Thread B when it attains the 'safe point'.  In this method, Thread B signals
-     * Thread A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()}
-     * is called by Thread A.
-     * @throws InterruptedException
+     * Called by Thread B when it attains the 'safe point'. In this method, Thread B signals Thread
+     * A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()} is called
+     * by Thread A.
      */
     void safePointAttained() throws InterruptedException {
       this.safePointAttainedLatch.countDown();
@@ -1644,8 +874,8 @@ public class FSHLog implements WAL {
     }
 
     /**
-     * Called by Thread A when it is done with the work it needs to do while Thread B is
-     * halted.  This will release the Thread B held in a call to {@link #safePointAttained()}
+     * Called by Thread A when it is done with the work it needs to do while Thread B is halted.
+     * This will release the Thread B held in a call to {@link #safePointAttained()}
      */
     void releaseSafePoint() {
       this.safePointReleasedLatch.countDown();
@@ -1655,44 +885,44 @@ public class FSHLog implements WAL {
      * @return True is this is a 'cocked', fresh instance, and not one that has already fired.
      */
     boolean isCocked() {
-      return this.safePointAttainedLatch.getCount() > 0 &&
-        this.safePointReleasedLatch.getCount() > 0;
+      return this.safePointAttainedLatch.getCount() > 0
+          && this.safePointReleasedLatch.getCount() > 0;
     }
   }
 
   /**
    * Handler that is run by the disruptor ringbuffer consumer. Consumer is a SINGLE
-   * 'writer/appender' thread.  Appends edits and starts up sync runs.  Tries its best to batch up
-   * syncs.  There is no discernible benefit batching appends so we just append as they come in
-   * because it simplifies the below implementation.  See metrics for batching effectiveness
-   * (In measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10
-   * handler sync invocations for every actual dfsclient sync call; at 10 concurrent handlers,
-   * YMMV).
-   * <p>Herein, we have an array into which we store the sync futures as they come in.  When we
-   * have a 'batch', we'll then pass what we have collected to a SyncRunner thread to do the
-   * filesystem sync.  When it completes, it will then call
-   * {@link SyncFuture#done(long, Throwable)} on each of SyncFutures in the batch to release
-   * blocked Handler threads.
-   * <p>I've tried various effects to try and make latencies low while keeping throughput high.
-   * I've tried keeping a single Queue of SyncFutures in this class appending to its tail as the
-   * syncs coming and having sync runner threads poll off the head to 'finish' completed
-   * SyncFutures.  I've tried linkedlist, and various from concurrent utils whether
-   * LinkedBlockingQueue or ArrayBlockingQueue, etc.  The more points of synchronization, the
-   * more 'work' (according to 'perf stats') that has to be done; small increases in stall
-   * percentages seem to have a big impact on throughput/latencies.  The below model where we have
-   * an array into which we stash the syncs and then hand them off to the sync thread seemed like
-   * a decent compromise.  See HBASE-8755 for more detail.
+   * 'writer/appender' thread. Appends edits and starts up sync runs. Tries its best to batch up
+   * syncs. There is no discernible benefit batching appends so we just append as they come in
+   * because it simplifies the below implementation. See metrics for batching effectiveness (In
+   * measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10 handler
+   * sync invocations for every actual dfsclient sync call; at 10 concurrent handlers, YMMV).
+   * <p>
+   * Herein, we have an array into which we store the sync futures as they come in. When we have a
+   * 'batch', we'll then pass what we have collected to a SyncRunner thread to do the filesystem
+   * sync. When it completes, it will then call {@link SyncFuture#done(long, Throwable)} on each of
+   * SyncFutures in the batch to release blocked Handler threads.
+   * <p>
+   * I've tried various effects to try and make latencies low while keeping throughput high. I've
+   * tried keeping a single Queue of SyncFutures in this class appending to its tail as the syncs
+   * coming and having sync runner threads poll off the head to 'finish' completed SyncFutures. I've
+   * tried linkedlist, and various from concurrent utils whether LinkedBlockingQueue or
+   * ArrayBlockingQueue, etc. The more points of synchronization, the more 'work' (according to
+   * 'perf stats') that has to be done; small increases in stall percentages seem to have a big
+   * impact on throughput/latencies. The below model where we have an array into which we stash the
+   * syncs and then hand them off to the sync thread seemed like a decent compromise. See HBASE-8755
+   * for more detail.
    */
   class RingBufferEventHandler implements EventHandler<RingBufferTruck>, LifecycleAware {
-    private final SyncRunner [] syncRunners;
-    private final SyncFuture [] syncFutures;
-    // Had 'interesting' issues when this was non-volatile.  On occasion, we'd not pass all
+    private final SyncRunner[] syncRunners;
+    private final SyncFuture[] syncFutures;
+    // Had 'interesting' issues when this was non-volatile. On occasion, we'd not pass all
     // syncFutures to the next sync'ing thread.
     private volatile int syncFuturesCount = 0;
     private volatile SafePointZigZagLatch zigzagLatch;
     /**
-     * Set if we get an exception appending or syncing so that all subsequence appends and syncs
-     * on this WAL fail until WAL is replaced.
+     * Set if we get an exception appending or syncing so that all subsequence appends and syncs on
+     * this WAL fail until WAL is replaced.
      */
     private Exception exception = null;
     /**
@@ -1716,7 +946,9 @@ public class FSHLog implements WAL {
 
     private void cleanupOutstandingSyncsOnException(final long sequence, final Exception e) {
       // There could be handler-count syncFutures outstanding.
-      for (int i = 0; i < this.syncFuturesCount; i++) this.syncFutures[i].done(sequence, e);
+      for (int i = 0; i < this.syncFuturesCount; i++) {
+        this.syncFutures[i].done(sequence, e);
+      }
       this.syncFuturesCount = 0;
     }
 
@@ -1725,7 +957,9 @@ public class FSHLog implements WAL {
      */
     private boolean isOutstandingSyncs() {
       for (int i = 0; i < this.syncFuturesCount; i++) {
-        if (!this.syncFutures[i].isDone()) return true;
+        if (!this.syncFutures[i].isDone()) {
+          return true;
+        }
       }
       return false;
     }
@@ -1733,10 +967,10 @@ public class FSHLog implements WAL {
     @Override
     // We can set endOfBatch in the below method if at end of our this.syncFutures array
     public void onEvent(final RingBufferTruck truck, final long sequence, boolean endOfBatch)
-    throws Exception {
-      // Appends and syncs are coming in order off the ringbuffer.  We depend on this fact.  We'll
-      // add appends to dfsclient as they come in.  Batching appends doesn't give any significant


<TRUNCATED>