You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2014/12/02 18:20:44 UTC

[05/21] hbase git commit: HBASE-12522 Backport of write-ahead-log refactoring and follow-ons.

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/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 cf70d4a..09b66bb 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
@@ -37,15 +37,16 @@ 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.FSDataOutputStream;
 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.LargeTests;
 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;
@@ -63,6 +64,9 @@ 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;
@@ -80,7 +84,6 @@ import org.junit.experimental.categories.Category;
 public class TestLogRolling  {
   private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
   private HRegionServer server;
-  private HLog log;
   private String tableName;
   private byte[] value;
   private FileSystem fs;
@@ -95,7 +98,6 @@ public class TestLogRolling  {
    */
   public TestLogRolling()  {
     this.server = null;
-    this.log = null;
     this.tableName = null;
 
     String className = this.getClass().getName();
@@ -174,12 +176,10 @@ public class TestLogRolling  {
     // When the hbase:meta table can be opened, the region servers are running
     new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
     this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
-    this.log = server.getWAL();
 
     Table table = createTestTable(this.tableName);
 
     server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
-    this.log = server.getWAL();
     for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
       doPut(table, i);
       if (i % 32 == 0) {
@@ -198,15 +198,15 @@ public class TestLogRolling  {
    */
   @Test(timeout=120000)
   public void testLogRollOnNothingWritten() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    HFileSystem fs = new HFileSystem(conf, false);
-    HLog newLog = HLogFactory.createHLog(fs.getBackingFs(),
-      FSUtils.getRootDir(conf), "test", conf, null, "test.com:8080:1");
+    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[]{});
     try {
       // Now roll the log before we write anything.
       newLog.rollWriter(true);
     } finally {
-      newLog.closeAndDelete();
+      wals.close();
     }
   }
 
@@ -220,7 +220,9 @@ public class TestLogRolling  {
     this.tableName = getName();
       // TODO: Why does this write data take for ever?
       startAndWriteData();
-      LOG.info("after writing there are " + ((FSHLog) log).getNumRolledLogFiles() + " log files");
+    final WAL log = server.getWAL(null);
+    LOG.info("after writing there are " + DefaultWALProvider.getNumRolledLogFiles(log) +
+        " log files");
 
       // flush all regions
 
@@ -233,9 +235,8 @@ public class TestLogRolling  {
       // Now roll the log
       log.rollWriter();
 
-      int count = ((FSHLog) log).getNumRolledLogFiles();
-      LOG.info("after flushing all regions and rolling logs there are " +
-                                      ((FSHLog) log).getNumRolledLogFiles() + " log files");
+    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);
   }
 
@@ -265,7 +266,7 @@ public class TestLogRolling  {
     LOG.info("Validated row " + row);
   }
 
-  void batchWriteAndWait(Table table, int start, boolean expect, int timeout)
+  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"
@@ -294,41 +295,18 @@ public class TestLogRolling  {
   }
 
   /**
-   * Give me the HDFS pipeline for this log file
-   */
-  DatanodeInfo[] getPipeline(HLog log) throws IllegalArgumentException,
-      IllegalAccessException, InvocationTargetException {
-    OutputStream stm = ((FSHLog) log).getOutputStream();
-    Method getPipeline = null;
-    for (Method m : stm.getClass().getDeclaredMethods()) {
-      if (m.getName().endsWith("getPipeline")) {
-        getPipeline = m;
-        getPipeline.setAccessible(true);
-        break;
-      }
-    }
-
-    assertTrue("Need DFSOutputStream.getPipeline() for this test",
-        null != getPipeline);
-    Object repl = getPipeline.invoke(stm, new Object[] {} /* NO_ARGS */);
-    return (DatanodeInfo[]) repl;
-  }
-
-
-  /**
    * 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 HLog file replication set to 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()));
 
     this.server = cluster.getRegionServer(0);
-    this.log = server.getWAL();
 
     // Create the test table and open it
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName()));
@@ -339,9 +317,8 @@ public class TestLogRolling  {
     assertTrue(table.isAutoFlush());
 
     server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
-    this.log = server.getWAL();
+    final FSHLog log = (FSHLog) server.getWAL(null);
 
-    assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas());
     // don't run this test without append support (HDFS-200 & HDFS-142)
     assertTrue("Need append support for this test", FSUtils
         .isAppendSupported(TEST_UTIL.getConfiguration()));
@@ -370,13 +347,14 @@ public class TestLogRolling  {
     writeData(table, 2);
 
     long curTime = System.currentTimeMillis();
-    long oldFilenum = ((FSHLog) log).getFilenum();
+    LOG.info("log.getCurrentFileName(): " + log.getCurrentFileName());
+    long oldFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
     assertTrue("Log should have a timestamp older than now",
         curTime > oldFilenum && oldFilenum != -1);
 
     assertTrue("The log shouldn't have rolled yet",
-      oldFilenum == ((FSHLog) log).getFilenum());
-    final DatanodeInfo[] pipeline = getPipeline(log);
+        oldFilenum == DefaultWALProvider.extractFileNumFromWAL(log));
+    final DatanodeInfo[] pipeline = log.getPipeLine();
     assertTrue(pipeline.length ==
         fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
 
@@ -386,7 +364,7 @@ public class TestLogRolling  {
 
     // this write should succeed, but trigger a log roll
     writeData(table, 2);
-    long newFilenum = ((FSHLog) log).getFilenum();
+    long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
 
     assertTrue("Missing datanode should've triggered a log roll",
         newFilenum > oldFilenum && newFilenum > curTime);
@@ -394,15 +372,15 @@ public class TestLogRolling  {
     // write some more log data (this should use a new hdfs_out)
     writeData(table, 3);
     assertTrue("The log should not roll again.",
-      ((FSHLog) log).getFilenum() == 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, 3, false, 14000);
+    batchWriteAndWait(table, log, 3, false, 14000);
+    int replication = log.getLogReplication();
     assertTrue("LowReplication Roller should've been disabled, current replication="
-            + ((FSHLog) log).getLogReplication(),
-        !log.isLowReplicationRollEnabled());
+            + replication, !log.isLowReplicationRollEnabled());
 
     dfsCluster
         .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
@@ -410,24 +388,23 @@ public class TestLogRolling  {
     // Force roll writer. The new log file will have the default replications,
     // and the LowReplication Roller will be enabled.
     log.rollWriter(true);
-    batchWriteAndWait(table, 13, true, 10000);
+    batchWriteAndWait(table, log, 13, true, 10000);
+    replication = log.getLogReplication();
     assertTrue("New log file should have the default replication instead of " +
-      ((FSHLog) log).getLogReplication(),
-      ((FSHLog) log).getLogReplication() ==
-        fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
-    assertTrue("LowReplication Roller should've been enabled",
-        log.isLowReplicationRollEnabled());
+      replication,
+      replication == fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
+    assertTrue("LowReplication Roller should've been enabled", log.isLowReplicationRollEnabled());
   }
 
   /**
-   * Test that HLog is rolled when all data nodes in the pipeline have been
+   * Test that WAL is rolled when all data nodes in the pipeline have been
    * restarted.
    * @throws Exception
    */
   @Test
   public void testLogRollOnPipelineRestart() throws Exception {
     LOG.info("Starting testLogRollOnPipelineRestart");
-    assertTrue("This test requires HLog file replication.",
+    assertTrue("This test requires WAL file replication.",
       fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1);
     LOG.info("Replication=" +
       fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()));
@@ -435,7 +412,6 @@ public class TestLogRolling  {
     Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
     try {
       this.server = cluster.getRegionServer(0);
-      this.log = server.getWAL();
 
       // Create the test table and open it
       HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(getName()));
@@ -445,11 +421,11 @@ public class TestLogRolling  {
       HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
 
       server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
-      this.log = server.getWAL();
+      final WAL log = server.getWAL(null);
       final List<Path> paths = new ArrayList<Path>();
       final List<Integer> preLogRolledCalled = new ArrayList<Integer>();
-      paths.add(((FSHLog) log).computeFilename());
-      log.registerWALActionsListener(new WALActionsListener() {
+      paths.add(DefaultWALProvider.getCurrentFileName(log));
+      log.registerWALActionsListener(new WALActionsListener.Base() {
         @Override
         public void preLogRoll(Path oldFile, Path newFile)  {
           LOG.debug("preLogRoll: oldFile="+oldFile+" newFile="+newFile);
@@ -459,23 +435,8 @@ public class TestLogRolling  {
         public void postLogRoll(Path oldFile, Path newFile) {
           paths.add(newFile);
         }
-        @Override
-        public void preLogArchive(Path oldFile, Path newFile) {}
-        @Override
-        public void postLogArchive(Path oldFile, Path newFile) {}
-        @Override
-        public void logRollRequested() {}
-        @Override
-        public void logCloseRequested() {}
-        @Override
-        public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
-            WALEdit logEdit) {}
-        @Override
-        public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
-            WALEdit logEdit) {}
       });
 
-      assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas());
       // don't run this test without append support (HDFS-200 & HDFS-142)
       assertTrue("Need append support for this test", FSUtils
           .isAppendSupported(TEST_UTIL.getConfiguration()));
@@ -485,11 +446,13 @@ public class TestLogRolling  {
       table.setAutoFlush(true, true);
 
       long curTime = System.currentTimeMillis();
-      long oldFilenum = log.getFilenum();
+      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);
 
-      assertTrue("The log shouldn't have rolled yet", oldFilenum == log.getFilenum());
+      assertTrue("The log shouldn't have rolled yet", oldFilenum ==
+          DefaultWALProvider.extractFileNumFromWAL(log));
 
       // roll all datanodes in the pipeline
       dfsCluster.restartDataNodes();
@@ -500,7 +463,7 @@ public class TestLogRolling  {
 
       // this write should succeed, but trigger a log roll
       writeData(table, 1003);
-      long newFilenum = log.getFilenum();
+      long newFilenum = DefaultWALProvider.extractFileNumFromWAL(log);
 
       assertTrue("Missing datanode should've triggered a log roll",
           newFilenum > oldFilenum && newFilenum > curTime);
@@ -531,12 +494,11 @@ public class TestLogRolling  {
         fsUtils.recoverFileLease(((HFileSystem)fs).getBackingFs(), p,
           TEST_UTIL.getConfiguration(), null);
 
-        LOG.debug("Reading HLog "+FSUtils.getPath(p));
-        HLog.Reader reader = null;
+        LOG.debug("Reading WAL "+FSUtils.getPath(p));
+        WAL.Reader reader = null;
         try {
-          reader = HLogFactory.createReader(fs, p,
-              TEST_UTIL.getConfiguration());
-          HLog.Entry entry;
+          reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration());
+          WAL.Entry entry;
           while ((entry = reader.next()) != null) {
             LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getCells());
             for (Cell cell : entry.getEdit().getCells()) {
@@ -600,8 +562,7 @@ public class TestLogRolling  {
       table2 = createTestTable(getName() + "1");
 
       server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
-      this.log = server.getWAL();
-      FSHLog fshLog = (FSHLog)log;
+      final WAL log = server.getWAL(null);
       HRegion region = server.getOnlineRegions(table2.getName()).get(0);
       Store s = region.getStore(HConstants.CATALOG_FAMILY);
 
@@ -614,12 +575,14 @@ public class TestLogRolling  {
         admin.flush(table2.getName());
       }
       doPut(table2, 3); // don't flush yet, or compaction might trigger before we roll WAL
-      assertEquals("Should have no WAL after initial writes", 0, fshLog.getNumRolledLogFiles());
+      assertEquals("Should have no WAL after initial writes", 0,
+          DefaultWALProvider.getNumRolledLogFiles(log));
       assertEquals(2, s.getStorefilesCount());
 
       // Roll the log and compact table2, to have compaction record in the 2nd WAL.
-      fshLog.rollWriter();
-      assertEquals("Should have WAL; one table is not flushed", 1, fshLog.getNumRolledLogFiles());
+      log.rollWriter();
+      assertEquals("Should have WAL; one table is not flushed", 1,
+          DefaultWALProvider.getNumRolledLogFiles(log));
       admin.flush(table2.getName());
       region.compactStores();
       // Wait for compaction in case if flush triggered it before us.
@@ -631,14 +594,16 @@ public class TestLogRolling  {
 
       // Write some value to the table so the WAL cannot be deleted until table is flushed.
       doPut(table, 0); // Now 2nd WAL will have compaction record for table2 and put for table.
-      fshLog.rollWriter(); // 1st WAL deleted, 2nd not deleted yet.
-      assertEquals("Should have WAL; one table is not flushed", 1, fshLog.getNumRolledLogFiles());
+      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);
-      fshLog.rollWriter(); // Now 2nd WAL is deleted and 3rd is added.
-      assertEquals("Should have 1 WALs at the end", 1, fshLog.getNumRolledLogFiles());
+      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();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index ea40f29..473c9ae 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -24,6 +24,7 @@ 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.HBaseTestingUtility;
@@ -34,11 +35,16 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+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.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
- * Test many concurrent appenders to an {@link #HLog} while rolling the log.
+ * Test many concurrent appenders to an {@link #WAL} while rolling the log.
  */
 @Category(SmallTests.class)
 public class TestLogRollingNoCluster {
@@ -58,8 +64,11 @@ public class TestLogRollingNoCluster {
     Path dir = TEST_UTIL.getDataTestDir();
     // The implementation needs to know the 'handler' count.
     TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, THREAD_COUNT);
-    HLog wal = HLogFactory.createHLog(fs, dir, "logs", TEST_UTIL.getConfiguration());
-
+    final Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    FSUtils.setRootDir(conf, dir);
+    final WALFactory wals = new WALFactory(conf, null, TestLogRollingNoCluster.class.getName());
+    final WAL wal = wals.getWAL(new byte[]{});
+    
     Appender [] appenders = null;
 
     final int count = THREAD_COUNT;
@@ -77,7 +86,7 @@ public class TestLogRollingNoCluster {
         appenders[i].join();
       }
     } finally {
-      wal.close();
+      wals.close();
     }
     for (int i = 0; i < count; i++) {
       assertFalse(appenders[i].isException());
@@ -89,11 +98,11 @@ public class TestLogRollingNoCluster {
    */
   static class Appender extends Thread {
     private final Log log;
-    private final HLog wal;
+    private final WAL wal;
     private final int count;
     private Exception e = null;
 
-    Appender(final HLog wal, final int index, final int count) {
+    Appender(final WAL wal, final int index, final int count) {
       super("" + index);
       this.wal = wal;
       this.count = count;
@@ -118,16 +127,19 @@ public class TestLogRollingNoCluster {
       try {
         for (int i = 0; i < this.count; i++) {
           long now = System.currentTimeMillis();
-          // Roll every ten edits if the log has anything in it.
-          if (i % 10 == 0 && ((FSHLog) this.wal).getNumEntries() > 0) {
+          // Roll every ten edits
+          if (i % 10 == 0) {
             this.wal.rollWriter();
           }
           WALEdit edit = new WALEdit();
           byte[] bytes = Bytes.toBytes(i);
           edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
-          this.wal.append(HRegionInfo.FIRST_META_REGIONINFO,
-              TableName.META_TABLE_NAME,
-              edit, now, HTableDescriptor.META_TABLEDESC, sequenceId);
+          final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
+          final FSTableDescriptors fts = new FSTableDescriptors(TEST_UTIL.getConfiguration());
+          final HTableDescriptor htd = fts.get(TableName.META_TABLE_NAME);
+          final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(),
+              TableName.META_TABLE_NAME, now), edit, sequenceId, true, null);
+          wal.sync(txid);
         }
         String msg = getName() + " finished";
         if (isException())

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/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
new file mode 100644
index 0000000..34f584c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
@@ -0,0 +1,208 @@
+/**
+ *
+ * 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.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.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.MediumTests;
+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.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.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * WAL tests that can be reused across providers.
+ */
+@Category(MediumTests.class)
+public class TestProtobufLog {
+  protected static final Log LOG = LogFactory.getLog(TestProtobufLog.class);
+
+  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);
+    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, val.getRow()));
+          String value = i + "" + idx;
+          assertArrayEquals(Bytes.toBytes(value), val.getValue());
+          idx++;
+        }
+      }
+      WAL.Entry entry = reader.next();
+      assertNull(entry);
+    } finally {
+      if (writer != null) {
+        writer.close();
+      }
+      if (reader != null) {
+        reader.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java
index 8e88f7b..dbf14a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestReadOldRootAndMetaEdits.java
@@ -40,6 +40,10 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.TableName;
 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.WALProvider;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -54,13 +58,15 @@ public class TestReadOldRootAndMetaEdits {
 
   private final static Log LOG = LogFactory.getLog(TestReadOldRootAndMetaEdits.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
   private static FileSystem fs;
   private static Path dir;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setClass("hbase.regionserver.hlog.writer.impl",
-      SequenceFileLogWriter.class, HLog.Writer.class);
+    conf = TEST_UTIL.getConfiguration();
+    conf.setClass("hbase.regionserver.hlog.writer.impl",
+      SequenceFileLogWriter.class, WALProvider.Writer.class);
     fs = TEST_UTIL.getTestFileSystem();
     dir = new Path(TEST_UTIL.createRootDir(), "testReadOldRootAndMetaEdits");
     fs.mkdirs(dir);
@@ -79,17 +85,14 @@ public class TestReadOldRootAndMetaEdits {
   @Test
   public void testReadOldRootAndMetaEdits() throws IOException {
     LOG.debug("testReadOldRootAndMetaEdits");
-    Configuration conf = HBaseConfiguration.create();
-    conf.setClass("hbase.regionserver.hlog.writer.impl", SequenceFileLogWriter.class,
-      HLog.Writer.class);
     // kv list to be used for all WALEdits.
     byte[] row = Bytes.toBytes("row");
     KeyValue kv = new KeyValue(row, row, row, row);
     List<KeyValue> kvs = new ArrayList<KeyValue>();
     kvs.add(kv);
 
-    HLog.Writer writer = null;
-    HLog.Reader reader = null;
+    WALProvider.Writer writer = null;
+    WAL.Reader reader = null;
     // a regular table
     TableName t = TableName.valueOf("t");
     HRegionInfo tRegionInfo = null;
@@ -98,21 +101,21 @@ public class TestReadOldRootAndMetaEdits {
     Path path = new Path(dir, "t");
     try {
       tRegionInfo = new HRegionInfo(t, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      HLog.Entry tEntry = createAEntry(new HLogKey(tRegionInfo.getEncodedNameAsBytes(), t,
+      WAL.Entry tEntry = createAEntry(new HLogKey(tRegionInfo.getEncodedNameAsBytes(), t,
           ++logCount, timestamp, HConstants.DEFAULT_CLUSTER_ID), kvs);
 
       // create a old root edit (-ROOT-).
-      HLog.Entry rootEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_ROOT_STR),
+      WAL.Entry rootEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_ROOT_STR),
           TableName.OLD_ROOT_TABLE_NAME, ++logCount, timestamp,
           HConstants.DEFAULT_CLUSTER_ID), kvs);
 
       // create a old meta edit (hbase:meta).
-      HLog.Entry oldMetaEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_META_STR),
+      WAL.Entry oldMetaEntry = createAEntry(new HLogKey(Bytes.toBytes(TableName.OLD_META_STR),
           TableName.OLD_META_TABLE_NAME, ++logCount, timestamp,
           HConstants.DEFAULT_CLUSTER_ID), kvs);
 
       // write above entries
-      writer = HLogFactory.createWALWriter(fs, path, conf);
+      writer = WALFactory.createWALWriter(fs, path, conf);
       writer.append(tEntry);
       writer.append(rootEntry);
       writer.append(oldMetaEntry);
@@ -122,8 +125,8 @@ public class TestReadOldRootAndMetaEdits {
       writer.close();
 
       // read the log and see things are okay.
-      reader = HLogFactory.createReader(fs, path, conf);
-      HLog.Entry entry = reader.next();
+      reader = WALFactory.createReader(fs, path, conf);
+      WAL.Entry entry = reader.next();
       assertNotNull(entry);
       assertTrue(entry.getKey().getTablename().equals(t));
       assertEquals(Bytes.toString(entry.getKey().getEncodedRegionName()),
@@ -144,15 +147,15 @@ public class TestReadOldRootAndMetaEdits {
     }
 }
   /**
-   * Creates a WALEdit for the passed KeyValues and returns a HLog.Entry instance composed of
-   * the WALEdit and passed HLogKey.
-   * @return HLog.Entry instance for the passed HLogKey and KeyValues
+   * Creates a WALEdit for the passed KeyValues and returns a WALProvider.Entry instance composed of
+   * the WALEdit and passed WALKey.
+   * @return WAL.Entry instance for the passed WALKey and KeyValues
    */
-  private HLog.Entry createAEntry(HLogKey hlogKey, List<KeyValue> kvs) {
+  private WAL.Entry createAEntry(WALKey walKey, List<KeyValue> kvs) {
     WALEdit edit = new WALEdit();
     for (KeyValue kv : kvs )
     edit.add(kv);
-    return new HLog.Entry(hlogKey, edit);
+    return new WAL.Entry(walKey, edit);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java
deleted file mode 100644
index 607149c..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureHLog.java
+++ /dev/null
@@ -1,129 +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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
-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.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.KeyValue;
-import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.log4j.Level;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(MediumTests.class)
-public class TestSecureHLog {
-  static final Log LOG = LogFactory.getLog(TestSecureHLog.class);
-  static {
-    ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hbase.regionserver.wal"))
-      .getLogger().setLevel(Level.ALL);
-  };
-  static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
-    conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
-    conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class,
-      HLog.Reader.class);
-    conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class,
-      HLog.Writer.class);
-    conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true);
-  }
-
-  @Test
-  public void testSecureHLog() throws Exception {
-    TableName tableName = TableName.valueOf("TestSecureHLog");
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor(tableName.getName()));
-    HRegionInfo regioninfo = new HRegionInfo(tableName,
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
-    final int total = 10;
-    final byte[] row = Bytes.toBytes("row");
-    final byte[] family = Bytes.toBytes("family");
-    final byte[] value = Bytes.toBytes("Test value");
-    FileSystem fs = TEST_UTIL.getTestFileSystem();
-    Path logDir = TEST_UTIL.getDataTestDir("log");
-    final AtomicLong sequenceId = new AtomicLong(1);
-
-    // Write the WAL
-    HLog wal = new FSHLog(fs, TEST_UTIL.getDataTestDir(), logDir.toString(),
-      TEST_UTIL.getConfiguration());
-    for (int i = 0; i < total; i++) {
-      WALEdit kvs = new WALEdit();
-      kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
-      wal.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd, sequenceId);
-    }
-    final Path walPath = ((FSHLog) wal).computeFilename();
-    wal.close();
-
-    // Insure edits are not plaintext
-    long length = fs.getFileStatus(walPath).getLen();
-    FSDataInputStream in = fs.open(walPath);
-    byte[] fileData = new byte[(int)length];
-    IOUtils.readFully(in, fileData);
-    in.close();
-    assertFalse("Cells appear to be plaintext", Bytes.contains(fileData, value));
-
-    // Confirm the WAL can be read back
-    HLog.Reader reader = HLogFactory.createReader(TEST_UTIL.getTestFileSystem(), walPath,
-      TEST_UTIL.getConfiguration());
-    int count = 0;
-    HLog.Entry entry = new HLog.Entry();
-    while (reader.next(entry) != null) {
-      count++;
-      List<Cell> cells = entry.getEdit().getCells();
-      assertTrue("Should be one KV per WALEdit", cells.size() == 1);
-      for (Cell cell: cells) {
-        byte[] thisRow = cell.getRow();
-        assertTrue("Incorrect row", Bytes.equals(thisRow, row));
-        byte[] thisFamily = cell.getFamily();
-        assertTrue("Incorrect family", Bytes.equals(thisFamily, family));
-        byte[] thisValue = cell.getValue();
-        assertTrue("Incorrect value", Bytes.equals(thisValue, value));
-      }
-    }
-    assertEquals("Should have read back as many KVs as written", total, count);
-    reader.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java
index 9dc6e2b..3fb8b86 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureWALReplay.java
@@ -21,6 +21,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
+import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
@@ -34,9 +36,9 @@ public class TestSecureWALReplay extends TestWALReplay {
     conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
     conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
     conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class,
-      HLog.Reader.class);
+      Reader.class);
     conf.setClass("hbase.regionserver.hlog.writer.impl", SecureProtobufLogWriter.class,
-      HLog.Writer.class);
+      Writer.class);
     conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true);
     TestWALReplay.setUpBeforeClass();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index 0a31d9f..6916748 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -29,6 +29,10 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -38,7 +42,7 @@ import org.junit.experimental.categories.Category;
 import static org.junit.Assert.*;
 
 /**
- * Test that the actions are called while playing with an HLog
+ * Test that the actions are called while playing with an WAL
  */
 @Category(SmallTests.class)
 public class TestWALActionsListener {
@@ -49,9 +53,6 @@ public class TestWALActionsListener {
 
   private final static byte[] SOME_BYTES =  Bytes.toBytes("t");
   private static FileSystem fs;
-  private static Path oldLogDir;
-  private static Path logDir;
-  private static String logName;
   private static Configuration conf;
 
   @BeforeClass
@@ -59,17 +60,13 @@ public class TestWALActionsListener {
     conf = TEST_UTIL.getConfiguration();
     conf.setInt("hbase.regionserver.maxlogs", 5);
     fs = FileSystem.get(conf);
-    oldLogDir = new Path(TEST_UTIL.getDataTestDir(),
-        HConstants.HREGION_OLDLOGDIR_NAME);
-    logName = HConstants.HREGION_LOGDIR_NAME;
-    logDir = new Path(TEST_UTIL.getDataTestDir(),
-        logName);
+    FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
   }
 
   @Before
   public void setUp() throws Exception {
-    fs.delete(logDir, true);
-    fs.delete(oldLogDir, true);
+    fs.delete(new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_LOGDIR_NAME), true);
+    fs.delete(new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_OLDLOGDIR_NAME), true);
   }
 
   @After
@@ -87,12 +84,12 @@ public class TestWALActionsListener {
     DummyWALActionsListener observer = new DummyWALActionsListener();
     List<WALActionsListener> list = new ArrayList<WALActionsListener>();
     list.add(observer);
+    final WALFactory wals = new WALFactory(conf, list, "testActionListener");
     DummyWALActionsListener laterobserver = new DummyWALActionsListener();
-    HLog hlog = HLogFactory.createHLog(fs, TEST_UTIL.getDataTestDir(), logName,
-                                       conf, list, null);
     final AtomicLong sequenceId = new AtomicLong(1);
     HRegionInfo hri = new HRegionInfo(TableName.valueOf(SOME_BYTES),
              SOME_BYTES, SOME_BYTES, false);
+    final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes());
 
     for (int i = 0; i < 20; i++) {
       byte[] b = Bytes.toBytes(i+"");
@@ -102,17 +99,18 @@ public class TestWALActionsListener {
       HTableDescriptor htd = new HTableDescriptor();
       htd.addFamily(new HColumnDescriptor(b));
 
-      hlog.append(hri, TableName.valueOf(b), edit, 0, htd, sequenceId);
+      final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(),
+          TableName.valueOf(b), 0), edit, sequenceId, true, null);
+      wal.sync(txid);
       if (i == 10) {
-        hlog.registerWALActionsListener(laterobserver);
+        wal.registerWALActionsListener(laterobserver);
       }
       if (i % 2 == 0) {
-        hlog.rollWriter();
+        wal.rollWriter();
       }
     }
 
-    hlog.close();
-    hlog.closeAndDelete();
+    wal.close();
 
     assertEquals(11, observer.preLogRollCounter);
     assertEquals(11, observer.postLogRollCounter);
@@ -125,7 +123,7 @@ public class TestWALActionsListener {
   /**
    * Just counts when methods are called
    */
-  static class DummyWALActionsListener implements WALActionsListener {
+  static class DummyWALActionsListener extends WALActionsListener.Base {
     public int preLogRollCounter = 0;
     public int postLogRollCounter = 0;
     public int closedCount = 0;
@@ -141,36 +139,9 @@ public class TestWALActionsListener {
     }
 
     @Override
-    public void preLogArchive(Path oldFile, Path newFile) {
-      // Not interested
-    }
-
-    @Override
-    public void postLogArchive(Path oldFile, Path newFile) {
-      // Not interested
-    }
-
-    @Override
-    public void logRollRequested() {
-      // Not interested
-    }
-
-    @Override
-    public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
-        WALEdit logEdit) {
-      // Not interested
-
-    }
-
-    @Override
     public void logCloseRequested() {
       closedCount++;
     }
-
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
-      //To change body of implemented methods use File | Settings | File Templates.
-    }
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index 7e129e2..c6c2b82 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.when;
@@ -79,11 +80,18 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.TestName;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
@@ -102,6 +110,10 @@ public class TestWALReplay {
   private FileSystem fs;
   private Configuration conf;
   private RecoveryMode mode;
+  private WALFactory wals;
+
+  @Rule
+  public final TestName currentTest = new TestName();
 
 
   @BeforeClass
@@ -128,17 +140,19 @@ public class TestWALReplay {
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
     this.hbaseRootDir = FSUtils.getRootDir(this.conf);
     this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    this.logName = HConstants.HREGION_LOGDIR_NAME;
+    this.logName = DefaultWALProvider.getWALDirectoryName(currentTest.getMethodName() + "-manual");
     this.logDir = new Path(this.hbaseRootDir, logName);
     if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
       TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
     }
     this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
         RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
+    this.wals = new WALFactory(conf, null, currentTest.getMethodName());
   }
 
   @After
   public void tearDown() throws Exception {
+    this.wals.close();
     TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
   }
 
@@ -272,7 +286,7 @@ public class TestWALReplay {
     HRegion.closeHRegion(region2);
     final byte [] rowName = tableName.getName();
 
-    HLog wal1 = createWAL(this.conf);
+    WAL wal1 = createWAL(this.conf);
     // Add 1k to each family.
     final int countPerFamily = 1000;
     final AtomicLong sequenceId = new AtomicLong(1);
@@ -280,19 +294,19 @@ public class TestWALReplay {
       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee,
           wal1, htd, sequenceId);
     }
-    wal1.close();
+    wal1.shutdown();
     runWALSplit(this.conf);
 
-    HLog wal2 = createWAL(this.conf);
+    WAL wal2 = createWAL(this.conf);
     // Add 1k to each family.
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
           ee, wal2, htd, sequenceId);
     }
-    wal2.close();
+    wal2.shutdown();
     runWALSplit(this.conf);
 
-    HLog wal3 = createWAL(this.conf);
+    WAL wal3 = createWAL(this.conf);
     try {
       HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3);
       long seqid = region.getOpenSeqNum();
@@ -306,7 +320,7 @@ public class TestWALReplay {
       // TODO: Scan all.
       region.close();
     } finally {
-      wal3.closeAndDelete();
+      wal3.close();
     }
   }
 
@@ -332,7 +346,7 @@ public class TestWALReplay {
     HRegion region2 = HRegion.createHRegion(hri,
         hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
-    HLog wal = createWAL(this.conf);
+    WAL wal = createWAL(this.conf);
     HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
 
     byte [] family = htd.getFamilies().iterator().next().getName();
@@ -359,7 +373,7 @@ public class TestWALReplay {
       @Override
       public Object run() throws Exception {
         runWALSplit(newConf);
-        HLog wal2 = createWAL(newConf);
+        WAL wal2 = createWAL(newConf);
 
         HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
           hbaseRootDir, hri, htd, wal2);
@@ -369,7 +383,7 @@ public class TestWALReplay {
 
         // I can't close wal1.  Its been appropriated when we split.
         region2.close();
-        wal2.closeAndDelete();
+        wal2.close();
         return null;
       }
     });
@@ -399,7 +413,7 @@ public class TestWALReplay {
     HRegion region2 = HRegion.createHRegion(hri,
         hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
-    HLog wal = createWAL(this.conf);
+    WAL wal = createWAL(this.conf);
     HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
 
     // Add an edit so something in the WAL
@@ -431,7 +445,7 @@ public class TestWALReplay {
       @Override
       public Object run() throws Exception {
         runWALSplit(newConf);
-        HLog wal2 = createWAL(newConf);
+        WAL wal2 = createWAL(newConf);
 
         HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
             hbaseRootDir, hri, htd, wal2);
@@ -441,7 +455,7 @@ public class TestWALReplay {
 
         // I can't close wal1.  Its been appropriated when we split.
         region2.close();
-        wal2.closeAndDelete();
+        wal2.close();
         return null;
       }
     });
@@ -475,7 +489,7 @@ public class TestWALReplay {
     // Write countPerFamily edits into the three families.  Do a flush on one
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
-    HLog wal = createWAL(this.conf);
+    WAL wal = createWAL(this.conf);
     HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
     long seqid = region.getOpenSeqNum();
     boolean first = true;
@@ -496,9 +510,9 @@ public class TestWALReplay {
     // replay of log has the correct effect, that our seqids are calculated correctly so
     // all edits in logs are seen as 'stale'/old.
     region.close(true);
-    wal.close();
+    wal.shutdown();
     runWALSplit(this.conf);
-    HLog wal2 = createWAL(this.conf);
+    WAL wal2 = createWAL(this.conf);
     HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2);
     long seqid2 = region2.getOpenSeqNum();
     assertTrue(seqid + result.size() < seqid2);
@@ -515,9 +529,6 @@ public class TestWALReplay {
     final Result result2 = region2.get(g);
     assertEquals(2 * result.size(), result2.size());
     wal2.sync();
-    // Set down maximum recovery so we dfsclient doesn't linger retrying something
-    // long gone.
-    HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal2).getOutputStream(), 1);
     final Configuration newConf = HBaseConfiguration.create(this.conf);
     User user = HBaseTestingUtility.getDifferentUser(newConf,
       tableName.getNameAsString());
@@ -527,7 +538,7 @@ public class TestWALReplay {
         runWALSplit(newConf);
         FileSystem newFS = FileSystem.get(newConf);
         // Make a new wal for new region open.
-        HLog wal3 = createWAL(newConf);
+        WAL wal3 = createWAL(newConf);
         final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
         HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) {
           @Override
@@ -546,7 +557,7 @@ public class TestWALReplay {
 
         // I can't close wal1.  Its been appropriated when we split.
         region3.close();
-        wal3.closeAndDelete();
+        wal3.close();
         return null;
       }
     });
@@ -588,7 +599,7 @@ public class TestWALReplay {
     // Write countPerFamily edits into the three families.  Do a flush on one
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
-    HLog wal = createWAL(this.conf);
+    WAL wal = createWAL(this.conf);
     HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
     long seqid = region.getOpenSeqNum();
     for (HColumnDescriptor hcd: htd.getFamilies()) {
@@ -604,7 +615,7 @@ public class TestWALReplay {
     // Let us flush the region
     region.flushcache();
     region.close(true);
-    wal.close();
+    wal.shutdown();
 
     // delete the store files in the second column family to simulate a failure
     // in between the flushcache();
@@ -621,7 +632,7 @@ public class TestWALReplay {
 
     // Let us try to split and recover
     runWALSplit(this.conf);
-    HLog wal2 = createWAL(this.conf);
+    WAL wal2 = createWAL(this.conf);
     HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal2);
     long seqid2 = region2.getOpenSeqNum();
     assertTrue(seqid + result.size() < seqid2);
@@ -667,11 +678,11 @@ public class TestWALReplay {
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
     region3.close();
-    region3.getLog().closeAndDelete();
+    region3.getWAL().close();
     // Write countPerFamily edits into the three families. Do a flush on one
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
-    HLog wal = createWAL(this.conf);
+    WAL wal = createWAL(this.conf);
     RegionServerServices rsServices = Mockito.mock(RegionServerServices.class);
     Mockito.doReturn(false).when(rsServices).isAborted();
     when(rsServices.getServerName()).thenReturn(ServerName.valueOf("foo", 10, 10));
@@ -724,11 +735,11 @@ public class TestWALReplay {
     }
 
     region.close(true);
-    wal.close();
+    wal.shutdown();
 
     // Let us try to split and recover
     runWALSplit(this.conf);
-    HLog wal2 = createWAL(this.conf);
+    WAL wal2 = createWAL(this.conf);
     Mockito.doReturn(false).when(rsServices).isAborted();
     HRegion region2 =
       HRegion.openHRegion(this.hbaseRootDir, hri, htd, wal2, this.conf, rsServices, null);
@@ -751,7 +762,7 @@ public class TestWALReplay {
   }
 
   /**
-   * Create an HRegion with the result of a HLog split and test we only see the
+   * Create an HRegion with the result of a WAL split and test we only see the
    * good edits
    * @throws Exception
    */
@@ -767,7 +778,7 @@ public class TestWALReplay {
     HRegion region2 = HRegion.createHRegion(hri,
             hbaseRootDir, this.conf, htd);
     HRegion.closeHRegion(region2);
-    final HLog wal = createWAL(this.conf);
+    final WAL wal = createWAL(this.conf);
     final byte[] rowName = tableName.getName();
     final byte[] regionName = hri.getEncodedNameAsBytes();
     final AtomicLong sequenceId = new AtomicLong(1);
@@ -788,20 +799,19 @@ public class TestWALReplay {
     long now = ee.currentTime();
     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
       now, rowName));
-    wal.append(hri, tableName, edit, now, htd, sequenceId);
+    wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId,
+        true, null);
 
     // Delete the c family to verify deletes make it over.
     edit = new WALEdit();
     now = ee.currentTime();
     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now,
       KeyValue.Type.DeleteFamily));
-    wal.append(hri, tableName, edit, now, htd, sequenceId);
+    wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now), edit, sequenceId,
+        true, null);
 
     // Sync.
     wal.sync();
-    // Set down maximum recovery so we dfsclient doesn't linger retrying something
-    // long gone.
-    HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal).getOutputStream(), 1);
     // Make a new conf and a new fs for the splitter to run on so we can take
     // over old wal.
     final Configuration newConf = HBaseConfiguration.create(this.conf);
@@ -815,14 +825,14 @@ public class TestWALReplay {
         // 100k seems to make for about 4 flushes during HRegion#initialize.
         newConf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 100);
         // Make a new wal for new region.
-        HLog newWal = createWAL(newConf);
+        WAL newWal = createWAL(newConf);
         final AtomicInteger flushcount = new AtomicInteger(0);
         try {
           final HRegion region =
               new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) {
             @Override
             protected FlushResult internalFlushcache(
-                final HLog wal, final long myseqid, MonitoredTask status)
+                final WAL wal, final long myseqid, MonitoredTask status)
             throws IOException {
               LOG.info("InternalFlushCache Invoked");
               FlushResult fs = super.internalFlushcache(wal, myseqid,
@@ -843,7 +853,7 @@ public class TestWALReplay {
             result.size());
           region.close();
         } finally {
-          newWal.closeAndDelete();
+          newWal.close();
         }
         return null;
       }
@@ -853,8 +863,7 @@ public class TestWALReplay {
   @Test
   // the following test is for HBASE-6065
   public void testSequentialEditLogSeqNum() throws IOException {
-    final TableName tableName =
-        TableName.valueOf("testSequentialEditLogSeqNum");
+    final TableName tableName = TableName.valueOf(currentTest.getMethodName());
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
     final Path basedir =
         FSUtils.getTableDir(this.hbaseRootDir, tableName);
@@ -863,8 +872,8 @@ public class TestWALReplay {
     final int countPerFamily = 10;
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
 
-    // Mock the HLog
-    MockHLog wal = createMockWAL(this.conf);
+    // Mock the WAL
+    MockWAL wal = createMockWAL();
 
     HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
     for (HColumnDescriptor hcd : htd.getFamilies()) {
@@ -883,10 +892,12 @@ public class TestWALReplay {
     // allow complete cache flush with the previous seq number got after first
     // set of edits.
     wal.completeCacheFlush(hri.getEncodedNameAsBytes());
-    wal.close();
-    FileStatus[] listStatus = this.fs.listStatus(wal.getDir());
-    HLogSplitter.splitLogFile(hbaseRootDir, listStatus[0],
-      this.fs, this.conf, null, null, null, mode);
+    wal.shutdown();
+    FileStatus[] listStatus = wal.getFiles();
+    assertNotNull(listStatus);
+    assertTrue(listStatus.length > 0);
+    WALSplitter.splitLogFile(hbaseRootDir, listStatus[0],
+        this.fs, this.conf, null, null, null, mode, wals);
     FileStatus[] listStatus1 = this.fs.listStatus(
         new Path(FSUtils.getTableDir(hbaseRootDir, tableName),
             new Path(hri.getEncodedName(), "recovered.edits")));
@@ -900,11 +911,12 @@ public class TestWALReplay {
         lastestSeqNumber, editCount);
   }
 
-  static class MockHLog extends FSHLog {
+  static class MockWAL extends FSHLog {
     boolean doCompleteCacheFlush = false;
 
-    public MockHLog(FileSystem fs, Path rootDir, String logName, Configuration conf) throws IOException {
-      super(fs, rootDir, logName, conf);
+    public MockWAL(FileSystem fs, Path rootDir, String logName, Configuration conf)
+        throws IOException {
+      super(fs, rootDir, logName, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
     }
 
     @Override
@@ -923,11 +935,11 @@ public class TestWALReplay {
     return htd;
   }
 
-  private MockHLog createMockWAL(Configuration conf) throws IOException {
-    MockHLog wal = new MockHLog(FileSystem.get(conf), hbaseRootDir, logName, conf);
+  private MockWAL createMockWAL() throws IOException {
+    MockWAL wal = new MockWAL(fs, hbaseRootDir, logName, conf);
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
-    HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal).getOutputStream(), 1);
+    HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
     return wal;
   }
 
@@ -968,7 +980,7 @@ public class TestWALReplay {
   }
 
   private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName,
-      final byte[] family, final int count, EnvironmentEdge ee, final HLog wal,
+      final byte[] family, final int count, EnvironmentEdge ee, final WAL wal,
       final HTableDescriptor htd, final AtomicLong sequenceId)
   throws IOException {
     String familyStr = Bytes.toString(family);
@@ -978,8 +990,10 @@ public class TestWALReplay {
       WALEdit edit = new WALEdit();
       edit.add(new KeyValue(rowName, family, qualifierBytes,
         ee.currentTime(), columnBytes));
-      wal.append(hri, tableName, edit, ee.currentTime(), htd, sequenceId);
+      wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, ee.currentTime()),
+          edit, sequenceId, true, null);
     }
+    wal.sync();
   }
 
   static List<Put> addRegionEdits (final byte [] rowName, final byte [] family,
@@ -1013,8 +1027,8 @@ public class TestWALReplay {
    * @throws IOException
    */
   private Path runWALSplit(final Configuration c) throws IOException {
-    List<Path> splits = HLogSplitter.split(
-      hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c);
+    List<Path> splits = WALSplitter.split(
+      hbaseRootDir, logDir, oldLogDir, FileSystem.get(c), c, wals);
     // Split should generate only 1 file since there's only 1 region
     assertEquals("splits=" + splits, 1, splits.size());
     // Make sure the file exists
@@ -1028,12 +1042,11 @@ public class TestWALReplay {
    * @return WAL with retries set down from 5 to 1 only.
    * @throws IOException
    */
-  private HLog createWAL(final Configuration c) throws IOException {
-    HLog wal = HLogFactory.createHLog(FileSystem.get(c),
-        hbaseRootDir, logName, c);
+  private WAL createWAL(final Configuration c) throws IOException {
+    FSHLog wal = new FSHLog(FileSystem.get(c), hbaseRootDir, logName, c);
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
-    HBaseTestingUtility.setMaxRecoveryErrorCount(((FSHLog) wal).getOutputStream(), 1);
+    HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
     return wal;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
index 1d065a9..d0b1bab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java
@@ -21,16 +21,21 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
@@ -38,6 +43,8 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -100,7 +107,7 @@ public class TestMultiSlaveReplication {
 
     utility2 = new HBaseTestingUtility(conf2);
     utility2.setZkCluster(miniZK);
-    new ZooKeeperWatcher(conf2, "cluster3", null, true);
+    new ZooKeeperWatcher(conf2, "cluster2", null, true);
 
     utility3 = new HBaseTestingUtility(conf3);
     utility3.setZkCluster(miniZK);
@@ -143,7 +150,8 @@ public class TestMultiSlaveReplication {
     putAndWait(row2, famName, htable1, htable2);
 
     // now roll the region server's logs
-    new HBaseAdmin(conf1).rollHLogWriter(master.getRegionServer(0).getServerName().toString());
+    rollWALAndWait(utility1, htable1.getName(), row2);
+
     // after the log was rolled put a new row
     putAndWait(row3, famName, htable1, htable2);
 
@@ -166,8 +174,7 @@ public class TestMultiSlaveReplication {
     p.add(famName, row, row);
     htable1.put(p);
     // now roll the logs again
-    new HBaseAdmin(conf1).rollHLogWriter(master.getRegionServer(0)
-        .getServerName().toString());
+    rollWALAndWait(utility1, htable1.getName(), row);
 
     // cleanup "row2", also conveniently use this to wait replication
     // to finish
@@ -188,6 +195,48 @@ public class TestMultiSlaveReplication {
     utility1.shutdownMiniCluster();
   }
 
+  private void rollWALAndWait(final HBaseTestingUtility utility, final TableName table,
+      final byte[] row) throws IOException {
+    final Admin admin = utility.getHBaseAdmin();
+    final MiniHBaseCluster cluster = utility.getMiniHBaseCluster();
+
+    // find the region that corresponds to the given row.
+    HRegion region = null;
+    for (HRegion candidate : cluster.getRegions(table)) {
+      if (HRegion.rowIsInRange(candidate.getRegionInfo(), row)) {
+        region = candidate;
+        break;
+      }
+    }
+    assertNotNull("Couldn't find the region for row '" + Arrays.toString(row) + "'", region);
+
+    final CountDownLatch latch = new CountDownLatch(1);
+
+    // listen for successful log rolls
+    final WALActionsListener listener = new WALActionsListener.Base() {
+          @Override
+          public void postLogRoll(final Path oldPath, final Path newPath) throws IOException {
+            latch.countDown();
+          }
+        };
+    region.getWAL().registerWALActionsListener(listener);
+
+    // request a roll
+    admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDesc().getTableName(),
+      region.getRegionName()));
+
+    // wait
+    try {
+      latch.await();
+    } catch (InterruptedException exception) {
+      LOG.warn("Interrupted while waiting for the wal of '" + region + "' to roll. If later " +
+          "replication tests fail, it's probably because we should still be waiting.");
+      Thread.currentThread().interrupt();
+    }
+    region.getWAL().unregisterWALActionsListener(listener);
+  }
+
+ 
   private void checkWithWait(byte[] row, int count, Table table) throws Exception {
     Get get = new Get(row);
     for (int i = 0; i < NB_RETRIES; i++) {
@@ -199,7 +248,8 @@ public class TestMultiSlaveReplication {
       if (res.size() >= 1) {
         LOG.info("Row is replicated");
         rowReplicated = true;
-        assertEquals(count, res.size());
+        assertEquals("Table '" + table + "' did not have the expected number of  results.",
+            count, res.size());
         break;
       }
       if (rowReplicated) {
@@ -214,7 +264,8 @@ public class TestMultiSlaveReplication {
     Get get = new Get(row);
     for (Table table : tables) {
       Result res = table.get(get);
-      assertEquals(count, res.size());
+      assertEquals("Table '" + table + "' did not have the expected number of results.",
+          count, res.size());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
index 64f0a35..76416f2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
@@ -59,7 +59,7 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
     // rolling like this makes sure the most recent one gets added to the queue
     for (JVMClusterUtil.RegionServerThread r :
                           utility1.getHBaseCluster().getRegionServerThreads()) {
-      r.getRegionServer().getWAL().rollWriter();
+      utility1.getHBaseAdmin().rollWALWriter(r.getRegionServer().getServerName());
     }
     utility1.truncateTable(tableName);
     // truncating the table will send one Delete per row to the slave cluster

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
index 00f3af6..82cc90d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
@@ -36,8 +36,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
@@ -81,7 +80,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     ReplicationEndpointReturningFalse.replicated.set(false);
     ReplicationEndpointForTest.lastEntries = null;
     for (RegionServerThread rs : utility1.getMiniHBaseCluster().getRegionServerThreads()) {
-      utility1.getHBaseAdmin().rollHLogWriter(rs.getRegionServer().getServerName().toString());
+      utility1.getHBaseAdmin().rollWALWriter(rs.getRegionServer().getServerName());
     }
   }
 
@@ -215,7 +214,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     static AtomicInteger startedCount = new AtomicInteger();
     static AtomicInteger stoppedCount = new AtomicInteger();
     static AtomicInteger replicateCount = new AtomicInteger();
-    static volatile List<HLog.Entry> lastEntries = null;
+    static volatile List<Entry> lastEntries = null;
 
     public ReplicationEndpointForTest() {
       contructedCount.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
index ec5a3ba..36cdb34 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillMasterRSCompressed.java
@@ -24,7 +24,7 @@ import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
 /**
- * Run the same test as TestReplicationKillMasterRS but with HLog compression enabled
+ * Run the same test as TestReplicationKillMasterRS but with WAL compression enabled
  * Do not add other tests in this class.
  */
 @Category(LargeTests.class)

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index fdc89ab..799963a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -73,7 +73,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     // rolling like this makes sure the most recent one gets added to the queue
     for ( JVMClusterUtil.RegionServerThread r :
         utility1.getHBaseCluster().getRegionServerThreads()) {
-      r.getRegionServer().getWAL().rollWriter();
+      utility1.getHBaseAdmin().rollWALWriter(r.getRegionServer().getServerName());
     }
     utility1.truncateTable(tableName);
     // truncating the table will send one Delete per row to the slave cluster
@@ -379,7 +379,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
 
   /**
    * Do a more intense version testSmallBatch, one  that will trigger
-   * hlog rolling and other non-trivial code paths
+   * wal rolling and other non-trivial code paths
    * @throws Exception
    */
   @Test(timeout=300000)
@@ -498,7 +498,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     HRegionInfo hri = new HRegionInfo(htable1.getName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
     WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
-    Replication.scopeWALEdits(htable1.getTableDescriptor(), new HLogKey(), edit);
+    Replication.scopeWALEdits(htable1.getTableDescriptor(), new WALKey(), edit);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
index 4c5ece5..726766a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
@@ -32,11 +32,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -60,6 +62,7 @@ public class TestReplicationSource {
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniDFSCluster(1);
     FS = TEST_UTIL.getDFSCluster().getFileSystem();
+    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), FS.getHomeDirectory());
     oldLogDir = new Path(FS.getHomeDirectory(),
         HConstants.HREGION_OLDLOGDIR_NAME);
     if (FS.exists(oldLogDir)) FS.delete(oldLogDir, true);
@@ -79,23 +82,22 @@ public class TestReplicationSource {
     Path logPath = new Path(logDir, "log");
     if (!FS.exists(logDir)) FS.mkdirs(logDir);
     if (!FS.exists(oldLogDir)) FS.mkdirs(oldLogDir);
-    HLog.Writer writer = HLogFactory.createWALWriter(FS,
-      logPath, conf);
+    WALProvider.Writer writer = WALFactory.createWALWriter(FS, logPath,
+        TEST_UTIL.getConfiguration());
     for(int i = 0; i < 3; i++) {
       byte[] b = Bytes.toBytes(Integer.toString(i));
       KeyValue kv = new KeyValue(b,b,b);
       WALEdit edit = new WALEdit();
       edit.add(kv);
-      HLogKey key = new HLogKey(b, TableName.valueOf(b), 0, 0,
+      WALKey key = new WALKey(b, TableName.valueOf(b), 0, 0,
           HConstants.DEFAULT_CLUSTER_ID);
-      writer.append(new HLog.Entry(key, edit));
+      writer.append(new WAL.Entry(key, edit));
       writer.sync();
     }
     writer.close();
 
-    HLog.Reader reader = HLogFactory.createReader(FS, 
-        logPath, conf);
-    HLog.Entry entry = reader.next();
+    WAL.Reader reader = WALFactory.createReader(FS, logPath, TEST_UTIL.getConfiguration());
+    WAL.Entry entry = reader.next();
     assertNotNull(entry);
 
     Path oldLogPath = new Path(oldLogDir, "log");
@@ -108,7 +110,7 @@ public class TestReplicationSource {
     entry = reader.next();
 
     assertNull(entry);
-
+    reader.close();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8959828f/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
index 05a8069..a83b936 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
@@ -30,10 +30,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -57,21 +56,21 @@ public class TestReplicationWALEntryFilters {
     SystemTableWALEntryFilter filter = new SystemTableWALEntryFilter();
 
     // meta
-    HLogKey key1 = new HLogKey( HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
-      HTableDescriptor.META_TABLEDESC.getTableName());
-    HLog.Entry metaEntry = new Entry(key1, null);
+    WALKey key1 = new WALKey( HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
+      TableName.META_TABLE_NAME);
+    Entry metaEntry = new Entry(key1, null);
 
     assertNull(filter.filter(metaEntry));
 
     // ns table
-    HLogKey key2 = new HLogKey(new byte[] {}, TableName.NAMESPACE_TABLE_NAME);
-    HLog.Entry nsEntry = new Entry(key2, null);
+    WALKey key2 = new WALKey(new byte[] {}, TableName.NAMESPACE_TABLE_NAME);
+    Entry nsEntry = new Entry(key2, null);
     assertNull(filter.filter(nsEntry));
 
     // user table
 
-    HLogKey key3 = new HLogKey(new byte[] {}, TableName.valueOf("foo"));
-    HLog.Entry userEntry = new Entry(key3, null);
+    WALKey key3 = new WALKey(new byte[] {}, TableName.valueOf("foo"));
+    Entry userEntry = new Entry(key3, null);
 
     assertEquals(userEntry, filter.filter(userEntry));
   }
@@ -80,10 +79,10 @@ public class TestReplicationWALEntryFilters {
   public void testScopeWALEntryFilter() {
     ScopeWALEntryFilter filter = new ScopeWALEntryFilter();
 
-    HLog.Entry userEntry = createEntry(a, b);
-    HLog.Entry userEntryA = createEntry(a);
-    HLog.Entry userEntryB = createEntry(b);
-    HLog.Entry userEntryEmpty = createEntry();
+    Entry userEntry = createEntry(a, b);
+    Entry userEntryA = createEntry(a);
+    Entry userEntryB = createEntry(b);
+    Entry userEntryEmpty = createEntry();
 
     // no scopes
     assertEquals(null, filter.filter(userEntry));
@@ -155,7 +154,7 @@ public class TestReplicationWALEntryFilters {
 
   @Test
   public void testChainWALEntryFilter() {
-    HLog.Entry userEntry = createEntry(a, b, c);
+    Entry userEntry = createEntry(a, b, c);
 
     ChainWALEntryFilter filter = new ChainWALEntryFilter(passFilter);
     assertEquals(createEntry(a,b,c), filter.filter(userEntry));
@@ -206,7 +205,7 @@ public class TestReplicationWALEntryFilters {
     ReplicationPeer peer = mock(ReplicationPeer.class);
 
     when(peer.getTableCFs()).thenReturn(null);
-    HLog.Entry userEntry = createEntry(a, b, c);
+    Entry userEntry = createEntry(a, b, c);
     TableCfWALEntryFilter filter = new TableCfWALEntryFilter(peer);
     assertEquals(createEntry(a,b,c), filter.filter(userEntry));
 
@@ -242,24 +241,24 @@ public class TestReplicationWALEntryFilters {
     assertEquals(createEntry(a,c), filter.filter(userEntry));
   }
 
-  private HLog.Entry createEntry(byte[]... kvs) {
-    HLogKey key1 = new HLogKey(new byte[] {}, TableName.valueOf("foo"));
+  private Entry createEntry(byte[]... kvs) {
+    WALKey key1 = new WALKey(new byte[] {}, TableName.valueOf("foo"));
     WALEdit edit1 = new WALEdit();
 
     for (byte[] kv : kvs) {
       edit1.add(new KeyValue(kv, kv, kv));
     }
-    return new HLog.Entry(key1, edit1);
+    return new Entry(key1, edit1);
   }
 
 
-  private void assertEquals(HLog.Entry e1, HLog.Entry e2) {
+  private void assertEquals(Entry e1, Entry e2) {
     Assert.assertEquals(e1 == null, e2 == null);
     if (e1 == null) {
       return;
     }
 
-    // do not compare HLogKeys
+    // do not compare WALKeys
 
     // compare kvs
     Assert.assertEquals(e1.getEdit() == null, e2.getEdit() == null);