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

[1/3] hbase git commit: HBASE-19024 Provide a configurable option to hsync WAL edits to the disk for better durability (Harshal Jain)

Repository: hbase
Updated Branches:
  refs/heads/branch-1 764798d99 -> f976b3a8a
  refs/heads/branch-2 fabb1d97c -> 4fcc1ac7f
  refs/heads/master e7cfdac7b -> 17ac2fe9c


HBASE-19024 Provide a configurable option to hsync WAL edits to the disk for better durability (Harshal Jain)


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

Branch: refs/heads/branch-2
Commit: 4fcc1ac7f732ea9239880a1381cde97121a0703d
Parents: fabb1d9
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Mar 20 16:59:25 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Mar 20 17:30:39 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/Durability.java  |   1 -
 .../hadoop/hbase/regionserver/HRegion.java      |  30 ++--
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  37 +++--
 .../regionserver/wal/ProtobufLogWriter.java     |  19 ++-
 .../hbase/regionserver/wal/SyncFuture.java      |  11 ++
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |  17 ++
 .../apache/hadoop/hbase/wal/WALProvider.java    |   2 +-
 .../regionserver/TestFailedAppendAndSync.java   |   4 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   4 +-
 .../hbase/regionserver/TestWALLockup.java       |  12 +-
 .../regionserver/wal/AbstractTestWALReplay.java |   2 +-
 .../wal/TestLogRollingNoCluster.java            |   4 +-
 .../hbase/regionserver/wal/TestProtobufLog.java |   2 +-
 .../regionserver/wal/TestWALDurability.java     | 161 +++++++++++++++++++
 .../replication/TestReplicationSource.java      |   2 +-
 .../apache/hadoop/hbase/wal/FaultyFSLog.java    |   7 +-
 .../apache/hadoop/hbase/wal/IOTestProvider.java |   4 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |   6 +-
 18 files changed, 276 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
index 6f510df..aaf0b5c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
@@ -50,7 +50,6 @@ public enum Durability {
   SYNC_WAL,
   /**
    * Write the Mutation to the WAL synchronously and force the entries to disk.
-   * (Note: this is currently not supported and will behave identical to {@link #SYNC_WAL})
    * See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313</a>
    */
   FSYNC_WAL

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 61e68ca..732d5e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -229,16 +229,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
   public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
 
-  public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE =
-      "hbase.regionserver.minibatch.size";
-  public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000;
-
   /**
    * This is the global default value for durability. All tables/mutations not
    * defining a durability or using USE_DEFAULT will default to this value.
    */
   private static final Durability DEFAULT_DURABILITY = Durability.SYNC_WAL;
 
+  public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE =
+      "hbase.regionserver.minibatch.size";
+  public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000;
+
+  public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync";
+  public static final boolean DEFAULT_WAL_HSYNC = false;
+
   final AtomicBoolean closed = new AtomicBoolean(false);
 
   /* Closing can take some time; use the closing flag if there is stuff we don't
@@ -794,11 +797,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
      */
     this.rowProcessorTimeout = conf.getLong(
         "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
-    this.regionDurability = htd.getDurability() == Durability.USE_DEFAULT ?
-        DEFAULT_DURABILITY : htd.getDurability();
 
     this.storeHotnessProtector = new StoreHotnessProtector(this, conf);
 
+    boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC);
+    /**
+     * This is the global default value for durability. All tables/mutations not defining a
+     * durability or using USE_DEFAULT will default to this value.
+     */
+    Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL;
+    this.regionDurability =
+        this.htableDescriptor.getDurability() == Durability.USE_DEFAULT ? defaultDurability :
+          this.htableDescriptor.getDurability();
+
     if (rsServices != null) {
       this.rsAccounting = this.rsServices.getRegionServerAccounting();
       // don't initialize coprocessors if not running within a regionserver
@@ -8350,10 +8361,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // nothing do to
         break;
       case SYNC_WAL:
+          this.wal.sync(txid, false);
+          break;
       case FSYNC_WAL:
-        // sync the WAL edit (SYNC and FSYNC treated the same for now)
-        this.wal.sync(txid);
-        break;
+          this.wal.sync(txid, true);
+          break;
       default:
         throw new RuntimeException("Unknown durability " + durability);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 b1ae1dc..d255985 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -129,6 +130,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered
   private final int minTolerableReplication;
 
+  private final boolean useHsync;
+
   // 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
@@ -212,6 +215,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
       5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2);
 
+    this.useHsync = conf.getBoolean(HRegion.WAL_HSYNC_CONF_KEY, HRegion.DEFAULT_WAL_HSYNC);
+
     // rollWriter sets this.hdfs_out if it can.
     rollWriter();
 
@@ -255,7 +260,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
     try {
-      nextWriter.sync();
+      nextWriter.sync(useHsync);
       postSync(System.nanoTime() - startTimeNanos, 0);
     } catch (IOException e) {
       // optimization failed, no need to abort here.
@@ -332,7 +337,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           // sequence and zigzagLatch will be set together
           assert sequence > 0L : "Failed to get sequence from ring buffer";
           TraceUtil.addTimelineAnnotation("awaiting safepoint");
-          syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence));
+          syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence, false));
         }
       } catch (FailedSyncBeforeLogCloseException e) {
         // If unflushed/unsynced entries on close, it is reason to abort.
@@ -577,7 +582,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           Throwable lastException = null;
           try {
             TraceUtil.addTimelineAnnotation("syncing writer");
-            writer.sync();
+            writer.sync(useHsync);
             TraceUtil.addTimelineAnnotation("writer synced");
             currentSequence = updateHighestSyncedSequence(currentSequence);
           } catch (IOException e) {
@@ -684,15 +689,15 @@ public class FSHLog extends AbstractFSWAL<Writer> {
     return this.disruptor.getRingBuffer().next();
   }
 
-  private SyncFuture publishSyncOnRingBuffer() {
+  private SyncFuture publishSyncOnRingBuffer(boolean forceSync) {
     long sequence = getSequenceOnRingBuffer();
-    return publishSyncOnRingBuffer(sequence);
+    return publishSyncOnRingBuffer(sequence, forceSync);
   }
 
   @VisibleForTesting
-  protected SyncFuture publishSyncOnRingBuffer(long sequence) {
+  protected SyncFuture publishSyncOnRingBuffer(long sequence, boolean forceSync) {
     // here we use ring buffer sequence as transaction id
-    SyncFuture syncFuture = getSyncFuture(sequence);
+    SyncFuture syncFuture = getSyncFuture(sequence).setForceSync(forceSync);
     try {
       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
       truck.load(syncFuture);
@@ -703,8 +708,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   // Sync all known transactions
-  private void publishSyncThenBlockOnCompletion(TraceScope scope) throws IOException {
-    SyncFuture syncFuture = publishSyncOnRingBuffer();
+  private void publishSyncThenBlockOnCompletion(TraceScope scope, boolean forceSync) throws IOException {
+    SyncFuture syncFuture = publishSyncOnRingBuffer(forceSync);
     blockOnSync(syncFuture);
   }
 
@@ -731,19 +736,29 @@ public class FSHLog extends AbstractFSWAL<Writer> {
 
   @Override
   public void sync() throws IOException {
+    sync(useHsync);
+  }
+
+  @Override
+  public void sync(boolean forceSync) throws IOException {
     try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
-      publishSyncThenBlockOnCompletion(scope);
+      publishSyncThenBlockOnCompletion(scope, forceSync);
     }
   }
 
   @Override
   public void sync(long txid) throws IOException {
+    sync(txid, useHsync);
+  }
+
+  @Override
+  public void sync(long txid, boolean forceSync) throws IOException {
     if (this.highestSyncedTxid.get() >= txid) {
       // Already sync'd.
       return;
     }
     try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
-      publishSyncThenBlockOnCompletion(scope);
+      publishSyncThenBlockOnCompletion(scope, forceSync);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 9d36429..2852047 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
@@ -72,11 +72,15 @@ public class ProtobufLogWriter extends AbstractProtobufLogWriter
   }
 
   @Override
-  public void sync() throws IOException {
+  public void sync(boolean forceSync) throws IOException {
     FSDataOutputStream fsdos = this.output;
     if (fsdos == null) return; // Presume closed
     fsdos.flush();
-    fsdos.hflush();
+    if (forceSync) {
+      fsdos.hsync();
+    } else {
+      fsdos.hflush();
+    }
   }
 
   public FSDataOutputStream getStream() {
@@ -89,10 +93,13 @@ public class ProtobufLogWriter extends AbstractProtobufLogWriter
       short replication, long blockSize) throws IOException, StreamLacksCapabilityException {
     this.output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize,
       null);
-    // TODO Be sure to add a check for hsync if this branch includes HBASE-19024
-    if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true) &&
-        !(CommonFSUtils.hasCapability(output, "hflush"))) {
-      throw new StreamLacksCapabilityException("hflush");
+    if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true)) {
+      if (!CommonFSUtils.hasCapability(output, "hflush")) {
+        throw new StreamLacksCapabilityException("hflush");
+      }
+      if (!CommonFSUtils.hasCapability(output, "hsync")) {
+        throw new StreamLacksCapabilityException("hsync");
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 bd30797..3e0998e 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
@@ -67,6 +67,8 @@ class SyncFuture {
 
   private Thread t;
 
+  private boolean forceSync;
+
   /**
    * Call this method to clear old usage and get it ready for new deploy.
    * @param txid the new transaction id
@@ -97,6 +99,15 @@ class SyncFuture {
     return this.txid;
   }
 
+  synchronized boolean isForceSync() {
+    return forceSync;
+  }
+
+  synchronized SyncFuture setForceSync(boolean forceSync) {
+    this.forceSync = forceSync;
+    return this;
+  }
+
   /**
    * @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).

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 db6c411..3c85737 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
@@ -140,6 +140,23 @@ public interface WAL extends Closeable, WALFileLengthProvider {
   void sync(long txid) throws IOException;
 
   /**
+   * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
+   *          vs hsync.
+   */
+  default void sync(boolean forceSync) throws IOException {
+    sync();
+  }
+
+  /**
+   * @param txid Transaction id to sync to.
+   * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
+   *          vs hsync.
+   */
+  default void sync(long txid, boolean forceSync) throws IOException {
+    sync(txid);
+  }
+
+  /**
    * WAL keeps track of the sequence numbers that are as yet not flushed im memstores
    * in order to be able to do accounting to figure which WALs can be let go. This method tells WAL
    * that some region is about to flush. The flush can be the whole region or for a column family

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 7ad815e..244a636 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
@@ -79,7 +79,7 @@ public interface WALProvider {
   // Writers are used internally. Users outside of the WAL should be relying on the
   // interface provided by WAL.
   interface Writer extends WriterBase {
-    void sync() throws IOException;
+    void sync(boolean forceSync) throws IOException;
 
     void append(WAL.Entry entry) throws IOException;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 7cb3e63..3cf06d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -138,11 +138,11 @@ public class TestFailedAppendAndSync {
             }
 
             @Override
-            public void sync() throws IOException {
+            public void sync(boolean forceSync) throws IOException {
               if (throwSyncException) {
                 throw new IOException("FAKE! Failed to replace a bad datanode...");
               }
-              w.sync();
+              w.sync(forceSync);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 31dfa2a..3272afa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -1132,8 +1132,8 @@ public class TestHRegion {
           }
 
           @Override
-          public void sync() throws IOException {
-            w.sync();
+          public void sync(boolean forceSync) throws IOException {
+            w.sync(forceSync);
           }
 
           @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index d6a6079..bbea064 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -172,11 +172,11 @@ public class TestWALLockup {
         }
 
         @Override
-        public void sync() throws IOException {
+        public void sync(boolean forceSync) throws IOException {
           if (throwException) {
             throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
           }
-          w.sync();
+          w.sync(forceSync);
         }
 
         @Override
@@ -252,7 +252,7 @@ public class TestWALLockup {
       dodgyWAL.append(region.getRegionInfo(), key, edit, true);
       boolean exception = false;
       try {
-        dodgyWAL.sync();
+        dodgyWAL.sync(false);
       } catch (Exception e) {
         exception = true;
       }
@@ -345,7 +345,7 @@ public class TestWALLockup {
 
       protected void publishSyncOnRingBufferAndBlock(long sequence) {
         try {
-          super.blockOnSync(super.publishSyncOnRingBuffer(sequence));
+          super.blockOnSync(super.publishSyncOnRingBuffer(sequence, false));
           Assert.fail("Expect an IOException here.");
         } catch (IOException ignore) {
           // Here, we will get an IOException.
@@ -362,7 +362,7 @@ public class TestWALLockup {
           }
 
           @Override
-          public void sync() throws IOException {
+          public void sync(boolean forceSync) throws IOException {
             throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
           }
 
@@ -425,7 +425,7 @@ public class TestWALLockup {
 
       try {
         LOG.info("Call sync for testing whether RingBufferEventHandler is hanging.");
-        dodgyWAL.sync(); // Should not get a hang here, otherwise we will see timeout in this test.
+        dodgyWAL.sync(false); // Should not get a hang here, otherwise we will see timeout in this test.
         Assert.fail("Expect an IOException here.");
       } catch (IOException ignore) {
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index 52abf59..79ae177 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -1230,7 +1230,7 @@ public abstract class AbstractTestWALReplay {
     for (FSWALEntry entry : entries) {
       writer.append(entry);
     }
-    writer.sync();
+    writer.sync(false);
     writer.close();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 5ee0dfa..819df67 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
@@ -67,9 +67,9 @@ public class TestLogRollingNoCluster {
   /** ProtobufLogWriter that simulates higher latencies in sync() call */
   public static class HighLatencySyncWriter extends  ProtobufLogWriter {
     @Override
-    public void sync() throws IOException {
+    public void sync(boolean forceSync) throws IOException {
       Threads.sleep(ThreadLocalRandom.current().nextInt(10));
-      super.sync();
+      super.sync(forceSync);
       Threads.sleep(ThreadLocalRandom.current().nextInt(10));
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 080b5be..2d938d4 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
@@ -48,6 +48,6 @@ public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer>
 
   @Override
   protected void sync(Writer writer) throws IOException {
-    writer.sync();
+    writer.sync(false);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
new file mode 100644
index 0000000..17f24e8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
@@ -0,0 +1,161 @@
+/**
+ * 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.assertNull;
+
+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.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Tests for WAL write durability - hflush vs hsync
+ */
+@Category({ MediumTests.class })
+public class TestWALDurability {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestWALDurability.class);
+
+  private static final String COLUMN_FAMILY = "MyCF";
+  private static final byte[] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private Configuration conf;
+  private String dir;
+  @Rule
+  public TestName name = new TestName();
+
+  // Test names
+  protected TableName tableName;
+
+  @Before
+  public void setup() throws IOException {
+    conf = TEST_UTIL.getConfiguration();
+    dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
+    tableName = TableName.valueOf(name.getMethodName());
+  }
+
+  @Test
+  public void testWALDurability() throws IOException {
+    class CustomFSLog extends FSHLog {
+      private Boolean syncFlag;
+
+      public CustomFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
+          throws IOException {
+        super(fs, root, logDir, conf);
+      }
+
+      @Override
+      public void sync(boolean forceSync) throws IOException {
+        syncFlag = forceSync;
+        super.sync(forceSync);
+      }
+
+      @Override
+      public void sync(long txid, boolean forceSync) throws IOException {
+        syncFlag = forceSync;
+        super.sync(txid, forceSync);
+      }
+
+      private void resetSyncFlag() {
+        this.syncFlag = null;
+      }
+
+    }
+    // global hbase.wal.hsync false, no override in put call - hflush
+    conf.set(HRegion.WAL_HSYNC_CONF_KEY, "false");
+    FileSystem fs = FileSystem.get(conf);
+    Path rootDir = new Path(dir + getName());
+    CustomFSLog customFSLog = new CustomFSLog(fs, rootDir, getName(), conf);
+    HRegion region = initHRegion(tableName, null, null, customFSLog);
+    byte[] bytes = Bytes.toBytes(getName());
+    Put put = new Put(bytes);
+    put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
+
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, false);
+
+    // global hbase.wal.hsync true, no override in put call
+    conf.set(HRegion.WAL_HSYNC_CONF_KEY, "true");
+    fs = FileSystem.get(conf);
+    customFSLog = new CustomFSLog(fs, rootDir, getName(), conf);
+    region = initHRegion(tableName, null, null, customFSLog);
+
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, true);
+
+    // global hbase.wal.hsync true, durability set in put call - fsync
+    put.setDurability(Durability.FSYNC_WAL);
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, true);
+
+    // global hbase.wal.hsync true, durability set in put call - sync
+    put = new Put(bytes);
+    put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
+    put.setDurability(Durability.SYNC_WAL);
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, false);
+
+    HBaseTestingUtility.closeRegionAndWAL(region);
+  }
+
+  private String getName() {
+    return name.getMethodName();
+  }
+
+  /**
+   * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
+   *         when done.
+   */
+  public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
+      throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+    return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.USE_DEFAULT,
+      wal, COLUMN_FAMILY_BYTES);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/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 b98ca7f..1bb361b 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
@@ -122,7 +122,7 @@ public class TestReplicationSource {
       WALKeyImpl key = new WALKeyImpl(b, TableName.valueOf(b), 0, 0,
           HConstants.DEFAULT_CLUSTER_ID);
       writer.append(new WAL.Entry(key, edit));
-      writer.sync();
+      writer.sync(false);
     }
     writer.close();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index ddd1113..c7f1c41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
@@ -51,10 +51,15 @@ public class FaultyFSLog extends FSHLog {
 
   @Override
   public void sync(long txid) throws IOException {
+    sync(txid, false);
+  }
+
+  @Override
+  public void sync(long txid, boolean forceSync) throws IOException {
     if (this.ft == FailureType.SYNC) {
       throw new IOException("sync");
     }
-    super.sync(txid);
+    super.sync(txid, forceSync);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index 7bd017d..a5f7dd3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -266,9 +266,9 @@ public class IOTestProvider implements WALProvider {
     }
 
     @Override
-    public void sync() throws IOException {
+    public void sync(boolean forceSync) throws IOException {
       if (doSyncs) {
-        super.sync();
+        super.sync(forceSync);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fcc1ac7/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index f5800df..030c99f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -1359,7 +1359,7 @@ public class TestWALSplit {
     WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), TABLE_NAME, 1,
         EnvironmentEdgeManager.currentTime(), HConstants.DEFAULT_CLUSTER_ID);
     w.append(new Entry(key, edit));
-    w.sync();
+    w.sync(false);
   }
 
   private static void appendRegionEvent(Writer w, String region) throws IOException {
@@ -1377,7 +1377,7 @@ public class TestWALSplit {
         HConstants.DEFAULT_CLUSTER_ID);
     w.append(
         new Entry(walKey, new WALEdit().add(kv)));
-    w.sync();
+    w.sync(false);
   }
 
   public static long appendEntry(Writer writer, TableName table, byte[] region,
@@ -1387,7 +1387,7 @@ public class TestWALSplit {
     LOG.info(Thread.currentThread().getName() + " append");
     writer.append(createTestEntry(table, region, row, family, qualifier, value, seq));
     LOG.info(Thread.currentThread().getName() + " sync");
-    writer.sync();
+    writer.sync(false);
     return seq;
   }
 


[3/3] hbase git commit: HBASE-19024 Provide a configurable option to hsync WAL edits to the disk for better durability (Harshal Jain)

Posted by ap...@apache.org.
HBASE-19024 Provide a configurable option to hsync WAL edits to the disk for better durability (Harshal Jain)


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

Branch: refs/heads/master
Commit: 17ac2fe9c13d2b567c6bc06690e1491dfba83090
Parents: e7cfdac
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Mar 20 16:59:25 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Mar 20 21:07:59 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/Durability.java  |   1 -
 .../hadoop/hbase/regionserver/HRegion.java      |  30 ++--
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  37 +++--
 .../regionserver/wal/ProtobufLogWriter.java     |  19 ++-
 .../hbase/regionserver/wal/SyncFuture.java      |  11 ++
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |  17 ++
 .../apache/hadoop/hbase/wal/WALProvider.java    |   2 +-
 .../regionserver/TestFailedAppendAndSync.java   |   4 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   4 +-
 .../hbase/regionserver/TestWALLockup.java       |  12 +-
 .../regionserver/wal/AbstractTestWALReplay.java |   2 +-
 .../wal/TestLogRollingNoCluster.java            |   4 +-
 .../hbase/regionserver/wal/TestProtobufLog.java |   2 +-
 .../regionserver/wal/TestWALDurability.java     | 161 +++++++++++++++++++
 .../replication/TestReplicationSource.java      |   2 +-
 .../replication/TestSerialReplication.java      |   2 +-
 .../apache/hadoop/hbase/wal/FaultyFSLog.java    |   7 +-
 .../apache/hadoop/hbase/wal/IOTestProvider.java |   4 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |   6 +-
 19 files changed, 277 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
index 6f510df..aaf0b5c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Durability.java
@@ -50,7 +50,6 @@ public enum Durability {
   SYNC_WAL,
   /**
    * Write the Mutation to the WAL synchronously and force the entries to disk.
-   * (Note: this is currently not supported and will behave identical to {@link #SYNC_WAL})
    * See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313</a>
    */
   FSYNC_WAL

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 9e9a721..9bab148 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -230,16 +230,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
   public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
 
-  public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE =
-      "hbase.regionserver.minibatch.size";
-  public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000;
-
   /**
    * This is the global default value for durability. All tables/mutations not
    * defining a durability or using USE_DEFAULT will default to this value.
    */
   private static final Durability DEFAULT_DURABILITY = Durability.SYNC_WAL;
 
+  public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE =
+      "hbase.regionserver.minibatch.size";
+  public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000;
+
+  public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync";
+  public static final boolean DEFAULT_WAL_HSYNC = false;
+
   final AtomicBoolean closed = new AtomicBoolean(false);
 
   /* Closing can take some time; use the closing flag if there is stuff we don't
@@ -795,11 +798,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
      */
     this.rowProcessorTimeout = conf.getLong(
         "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
-    this.regionDurability = htd.getDurability() == Durability.USE_DEFAULT ?
-        DEFAULT_DURABILITY : htd.getDurability();
 
     this.storeHotnessProtector = new StoreHotnessProtector(this, conf);
 
+    boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC);
+    /**
+     * This is the global default value for durability. All tables/mutations not defining a
+     * durability or using USE_DEFAULT will default to this value.
+     */
+    Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL;
+    this.regionDurability =
+        this.htableDescriptor.getDurability() == Durability.USE_DEFAULT ? defaultDurability :
+          this.htableDescriptor.getDurability();
+
     if (rsServices != null) {
       this.rsAccounting = this.rsServices.getRegionServerAccounting();
       // don't initialize coprocessors if not running within a regionserver
@@ -8361,10 +8372,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // nothing do to
         break;
       case SYNC_WAL:
+          this.wal.sync(txid, false);
+          break;
       case FSYNC_WAL:
-        // sync the WAL edit (SYNC and FSYNC treated the same for now)
-        this.wal.sync(txid);
-        break;
+          this.wal.sync(txid, true);
+          break;
       default:
         throw new RuntimeException("Unknown durability " + durability);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 6b77e80..5b8feae 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -129,6 +130,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered
   private final int minTolerableReplication;
 
+  private final boolean useHsync;
+
   // 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
@@ -212,6 +215,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
       5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2);
 
+    this.useHsync = conf.getBoolean(HRegion.WAL_HSYNC_CONF_KEY, HRegion.DEFAULT_WAL_HSYNC);
+
     // rollWriter sets this.hdfs_out if it can.
     rollWriter();
 
@@ -255,7 +260,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
     try {
-      nextWriter.sync();
+      nextWriter.sync(useHsync);
       postSync(System.nanoTime() - startTimeNanos, 0);
     } catch (IOException e) {
       // optimization failed, no need to abort here.
@@ -332,7 +337,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           // sequence and zigzagLatch will be set together
           assert sequence > 0L : "Failed to get sequence from ring buffer";
           TraceUtil.addTimelineAnnotation("awaiting safepoint");
-          syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence));
+          syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence, false));
         }
       } catch (FailedSyncBeforeLogCloseException e) {
         // If unflushed/unsynced entries on close, it is reason to abort.
@@ -577,7 +582,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           Throwable lastException = null;
           try {
             TraceUtil.addTimelineAnnotation("syncing writer");
-            writer.sync();
+            writer.sync(useHsync);
             TraceUtil.addTimelineAnnotation("writer synced");
             currentSequence = updateHighestSyncedSequence(currentSequence);
           } catch (IOException e) {
@@ -684,15 +689,15 @@ public class FSHLog extends AbstractFSWAL<Writer> {
     return this.disruptor.getRingBuffer().next();
   }
 
-  private SyncFuture publishSyncOnRingBuffer() {
+  private SyncFuture publishSyncOnRingBuffer(boolean forceSync) {
     long sequence = getSequenceOnRingBuffer();
-    return publishSyncOnRingBuffer(sequence);
+    return publishSyncOnRingBuffer(sequence, forceSync);
   }
 
   @VisibleForTesting
-  protected SyncFuture publishSyncOnRingBuffer(long sequence) {
+  protected SyncFuture publishSyncOnRingBuffer(long sequence, boolean forceSync) {
     // here we use ring buffer sequence as transaction id
-    SyncFuture syncFuture = getSyncFuture(sequence);
+    SyncFuture syncFuture = getSyncFuture(sequence).setForceSync(forceSync);
     try {
       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
       truck.load(syncFuture);
@@ -703,8 +708,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   // Sync all known transactions
-  private void publishSyncThenBlockOnCompletion(TraceScope scope) throws IOException {
-    SyncFuture syncFuture = publishSyncOnRingBuffer();
+  private void publishSyncThenBlockOnCompletion(TraceScope scope, boolean forceSync) throws IOException {
+    SyncFuture syncFuture = publishSyncOnRingBuffer(forceSync);
     blockOnSync(syncFuture);
   }
 
@@ -731,19 +736,29 @@ public class FSHLog extends AbstractFSWAL<Writer> {
 
   @Override
   public void sync() throws IOException {
+    sync(useHsync);
+  }
+
+  @Override
+  public void sync(boolean forceSync) throws IOException {
     try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
-      publishSyncThenBlockOnCompletion(scope);
+      publishSyncThenBlockOnCompletion(scope, forceSync);
     }
   }
 
   @Override
   public void sync(long txid) throws IOException {
+    sync(txid, useHsync);
+  }
+
+  @Override
+  public void sync(long txid, boolean forceSync) throws IOException {
     if (this.highestSyncedTxid.get() >= txid) {
       // Already sync'd.
       return;
     }
     try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
-      publishSyncThenBlockOnCompletion(scope);
+      publishSyncThenBlockOnCompletion(scope, forceSync);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 9d36429..2852047 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
@@ -72,11 +72,15 @@ public class ProtobufLogWriter extends AbstractProtobufLogWriter
   }
 
   @Override
-  public void sync() throws IOException {
+  public void sync(boolean forceSync) throws IOException {
     FSDataOutputStream fsdos = this.output;
     if (fsdos == null) return; // Presume closed
     fsdos.flush();
-    fsdos.hflush();
+    if (forceSync) {
+      fsdos.hsync();
+    } else {
+      fsdos.hflush();
+    }
   }
 
   public FSDataOutputStream getStream() {
@@ -89,10 +93,13 @@ public class ProtobufLogWriter extends AbstractProtobufLogWriter
       short replication, long blockSize) throws IOException, StreamLacksCapabilityException {
     this.output = fs.createNonRecursive(path, overwritable, bufferSize, replication, blockSize,
       null);
-    // TODO Be sure to add a check for hsync if this branch includes HBASE-19024
-    if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true) &&
-        !(CommonFSUtils.hasCapability(output, "hflush"))) {
-      throw new StreamLacksCapabilityException("hflush");
+    if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true)) {
+      if (!CommonFSUtils.hasCapability(output, "hflush")) {
+        throw new StreamLacksCapabilityException("hflush");
+      }
+      if (!CommonFSUtils.hasCapability(output, "hsync")) {
+        throw new StreamLacksCapabilityException("hsync");
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 bd30797..3e0998e 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
@@ -67,6 +67,8 @@ class SyncFuture {
 
   private Thread t;
 
+  private boolean forceSync;
+
   /**
    * Call this method to clear old usage and get it ready for new deploy.
    * @param txid the new transaction id
@@ -97,6 +99,15 @@ class SyncFuture {
     return this.txid;
   }
 
+  synchronized boolean isForceSync() {
+    return forceSync;
+  }
+
+  synchronized SyncFuture setForceSync(boolean forceSync) {
+    this.forceSync = forceSync;
+    return this;
+  }
+
   /**
    * @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).

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 db6c411..3c85737 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
@@ -140,6 +140,23 @@ public interface WAL extends Closeable, WALFileLengthProvider {
   void sync(long txid) throws IOException;
 
   /**
+   * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
+   *          vs hsync.
+   */
+  default void sync(boolean forceSync) throws IOException {
+    sync();
+  }
+
+  /**
+   * @param txid Transaction id to sync to.
+   * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
+   *          vs hsync.
+   */
+  default void sync(long txid, boolean forceSync) throws IOException {
+    sync(txid);
+  }
+
+  /**
    * WAL keeps track of the sequence numbers that are as yet not flushed im memstores
    * in order to be able to do accounting to figure which WALs can be let go. This method tells WAL
    * that some region is about to flush. The flush can be the whole region or for a column family

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 7ad815e..244a636 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
@@ -79,7 +79,7 @@ public interface WALProvider {
   // Writers are used internally. Users outside of the WAL should be relying on the
   // interface provided by WAL.
   interface Writer extends WriterBase {
-    void sync() throws IOException;
+    void sync(boolean forceSync) throws IOException;
 
     void append(WAL.Entry entry) throws IOException;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 7cb3e63..3cf06d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -138,11 +138,11 @@ public class TestFailedAppendAndSync {
             }
 
             @Override
-            public void sync() throws IOException {
+            public void sync(boolean forceSync) throws IOException {
               if (throwSyncException) {
                 throw new IOException("FAKE! Failed to replace a bad datanode...");
               }
-              w.sync();
+              w.sync(forceSync);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 31dfa2a..3272afa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -1132,8 +1132,8 @@ public class TestHRegion {
           }
 
           @Override
-          public void sync() throws IOException {
-            w.sync();
+          public void sync(boolean forceSync) throws IOException {
+            w.sync(forceSync);
           }
 
           @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 9e051ae..29a75b8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -172,11 +172,11 @@ public class TestWALLockup {
         }
 
         @Override
-        public void sync() throws IOException {
+        public void sync(boolean forceSync) throws IOException {
           if (throwException) {
             throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
           }
-          w.sync();
+          w.sync(forceSync);
         }
 
         @Override
@@ -252,7 +252,7 @@ public class TestWALLockup {
       dodgyWAL.append(region.getRegionInfo(), key, edit, true);
       boolean exception = false;
       try {
-        dodgyWAL.sync();
+        dodgyWAL.sync(false);
       } catch (Exception e) {
         exception = true;
       }
@@ -345,7 +345,7 @@ public class TestWALLockup {
 
       protected void publishSyncOnRingBufferAndBlock(long sequence) {
         try {
-          super.blockOnSync(super.publishSyncOnRingBuffer(sequence));
+          super.blockOnSync(super.publishSyncOnRingBuffer(sequence, false));
           Assert.fail("Expect an IOException here.");
         } catch (IOException ignore) {
           // Here, we will get an IOException.
@@ -362,7 +362,7 @@ public class TestWALLockup {
           }
 
           @Override
-          public void sync() throws IOException {
+          public void sync(boolean forceSync) throws IOException {
             throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
           }
 
@@ -425,7 +425,7 @@ public class TestWALLockup {
 
       try {
         LOG.info("Call sync for testing whether RingBufferEventHandler is hanging.");
-        dodgyWAL.sync(); // Should not get a hang here, otherwise we will see timeout in this test.
+        dodgyWAL.sync(false); // Should not get a hang here, otherwise we will see timeout in this test.
         Assert.fail("Expect an IOException here.");
       } catch (IOException ignore) {
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index b1e304e..2d4b385 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -1230,7 +1230,7 @@ public abstract class AbstractTestWALReplay {
     for (FSWALEntry entry : entries) {
       writer.append(entry);
     }
-    writer.sync();
+    writer.sync(false);
     writer.close();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 5ee0dfa..819df67 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
@@ -67,9 +67,9 @@ public class TestLogRollingNoCluster {
   /** ProtobufLogWriter that simulates higher latencies in sync() call */
   public static class HighLatencySyncWriter extends  ProtobufLogWriter {
     @Override
-    public void sync() throws IOException {
+    public void sync(boolean forceSync) throws IOException {
       Threads.sleep(ThreadLocalRandom.current().nextInt(10));
-      super.sync();
+      super.sync(forceSync);
       Threads.sleep(ThreadLocalRandom.current().nextInt(10));
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 080b5be..2d938d4 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
@@ -48,6 +48,6 @@ public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer>
 
   @Override
   protected void sync(Writer writer) throws IOException {
-    writer.sync();
+    writer.sync(false);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
new file mode 100644
index 0000000..17f24e8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java
@@ -0,0 +1,161 @@
+/**
+ * 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.assertNull;
+
+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.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Tests for WAL write durability - hflush vs hsync
+ */
+@Category({ MediumTests.class })
+public class TestWALDurability {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestWALDurability.class);
+
+  private static final String COLUMN_FAMILY = "MyCF";
+  private static final byte[] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private Configuration conf;
+  private String dir;
+  @Rule
+  public TestName name = new TestName();
+
+  // Test names
+  protected TableName tableName;
+
+  @Before
+  public void setup() throws IOException {
+    conf = TEST_UTIL.getConfiguration();
+    dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
+    tableName = TableName.valueOf(name.getMethodName());
+  }
+
+  @Test
+  public void testWALDurability() throws IOException {
+    class CustomFSLog extends FSHLog {
+      private Boolean syncFlag;
+
+      public CustomFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
+          throws IOException {
+        super(fs, root, logDir, conf);
+      }
+
+      @Override
+      public void sync(boolean forceSync) throws IOException {
+        syncFlag = forceSync;
+        super.sync(forceSync);
+      }
+
+      @Override
+      public void sync(long txid, boolean forceSync) throws IOException {
+        syncFlag = forceSync;
+        super.sync(txid, forceSync);
+      }
+
+      private void resetSyncFlag() {
+        this.syncFlag = null;
+      }
+
+    }
+    // global hbase.wal.hsync false, no override in put call - hflush
+    conf.set(HRegion.WAL_HSYNC_CONF_KEY, "false");
+    FileSystem fs = FileSystem.get(conf);
+    Path rootDir = new Path(dir + getName());
+    CustomFSLog customFSLog = new CustomFSLog(fs, rootDir, getName(), conf);
+    HRegion region = initHRegion(tableName, null, null, customFSLog);
+    byte[] bytes = Bytes.toBytes(getName());
+    Put put = new Put(bytes);
+    put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
+
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, false);
+
+    // global hbase.wal.hsync true, no override in put call
+    conf.set(HRegion.WAL_HSYNC_CONF_KEY, "true");
+    fs = FileSystem.get(conf);
+    customFSLog = new CustomFSLog(fs, rootDir, getName(), conf);
+    region = initHRegion(tableName, null, null, customFSLog);
+
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, true);
+
+    // global hbase.wal.hsync true, durability set in put call - fsync
+    put.setDurability(Durability.FSYNC_WAL);
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, true);
+
+    // global hbase.wal.hsync true, durability set in put call - sync
+    put = new Put(bytes);
+    put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
+    put.setDurability(Durability.SYNC_WAL);
+    customFSLog.resetSyncFlag();
+    assertNull(customFSLog.syncFlag);
+    region.put(put);
+    assertEquals(customFSLog.syncFlag, false);
+
+    HBaseTestingUtility.closeRegionAndWAL(region);
+  }
+
+  private String getName() {
+    return name.getMethodName();
+  }
+
+  /**
+   * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
+   *         when done.
+   */
+  public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey, WAL wal)
+      throws IOException {
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+    return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, Durability.USE_DEFAULT,
+      wal, COLUMN_FAMILY_BYTES);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/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 b98ca7f..1bb361b 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
@@ -122,7 +122,7 @@ public class TestReplicationSource {
       WALKeyImpl key = new WALKeyImpl(b, TableName.valueOf(b), 0, 0,
           HConstants.DEFAULT_CLUSTER_ID);
       writer.append(new WAL.Entry(key, edit));
-      writer.sync();
+      writer.sync(false);
     }
     writer.close();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 4e8a4cc..7b440ce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -99,7 +99,7 @@ public class TestSerialReplication {
           for (Entry entry : replicateContext.getEntries()) {
             WRITER.append(entry);
           }
-          WRITER.sync();
+          WRITER.sync(false);
         } catch (IOException e) {
           throw new UncheckedIOException(e);
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index ddd1113..c7f1c41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
@@ -51,10 +51,15 @@ public class FaultyFSLog extends FSHLog {
 
   @Override
   public void sync(long txid) throws IOException {
+    sync(txid, false);
+  }
+
+  @Override
+  public void sync(long txid, boolean forceSync) throws IOException {
     if (this.ft == FailureType.SYNC) {
       throw new IOException("sync");
     }
-    super.sync(txid);
+    super.sync(txid, forceSync);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index 3928d9c..e54f1f8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -266,9 +266,9 @@ public class IOTestProvider implements WALProvider {
     }
 
     @Override
-    public void sync() throws IOException {
+    public void sync(boolean forceSync) throws IOException {
       if (doSyncs) {
-        super.sync();
+        super.sync(forceSync);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/17ac2fe9/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index f5800df..030c99f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -1359,7 +1359,7 @@ public class TestWALSplit {
     WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), TABLE_NAME, 1,
         EnvironmentEdgeManager.currentTime(), HConstants.DEFAULT_CLUSTER_ID);
     w.append(new Entry(key, edit));
-    w.sync();
+    w.sync(false);
   }
 
   private static void appendRegionEvent(Writer w, String region) throws IOException {
@@ -1377,7 +1377,7 @@ public class TestWALSplit {
         HConstants.DEFAULT_CLUSTER_ID);
     w.append(
         new Entry(walKey, new WALEdit().add(kv)));
-    w.sync();
+    w.sync(false);
   }
 
   public static long appendEntry(Writer writer, TableName table, byte[] region,
@@ -1387,7 +1387,7 @@ public class TestWALSplit {
     LOG.info(Thread.currentThread().getName() + " append");
     writer.append(createTestEntry(table, region, row, family, qualifier, value, seq));
     LOG.info(Thread.currentThread().getName() + " sync");
-    writer.sync();
+    writer.sync(false);
     return seq;
   }
 


[2/3] hbase git commit: HBASE-19024 Provide a configurable option to hsync WAL edits to the disk for better durability (Harshal Jain)

Posted by ap...@apache.org.
HBASE-19024 Provide a configurable option to hsync WAL edits to the disk for better durability (Harshal Jain)


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

Branch: refs/heads/branch-1
Commit: f976b3a8af86809b1f9f8b72f0dc67ef059f76bd
Parents: 764798d
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Mar 20 17:39:03 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Mar 20 17:51:59 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 20 ++++++++--
 .../hadoop/hbase/regionserver/wal/FSHLog.java   | 39 ++++++++++++++------
 .../regionserver/wal/ProtobufLogWriter.java     |  8 +++-
 .../hbase/regionserver/wal/SyncFuture.java      | 11 ++++++
 .../hadoop/hbase/wal/DisabledWALProvider.java   | 10 +++++
 .../java/org/apache/hadoop/hbase/wal/WAL.java   | 15 ++++++++
 .../apache/hadoop/hbase/wal/WALProvider.java    |  7 +++-
 .../hbase/client/TestRollbackFromClient.java    | 13 ++++++-
 .../regionserver/TestFailedAppendAndSync.java   | 12 +++---
 .../hadoop/hbase/regionserver/TestHRegion.java  |  8 ++--
 .../hbase/regionserver/TestWALLockup.java       | 11 +++---
 .../regionserver/wal/SequenceFileLogWriter.java |  3 +-
 .../wal/TestLogRollingNoCluster.java            |  4 +-
 .../hbase/regionserver/wal/TestProtobufLog.java |  2 +-
 .../wal/TestReadOldRootAndMetaEdits.java        |  2 +-
 .../hbase/regionserver/wal/TestWALReplay.java   |  2 +-
 .../replication/TestReplicationSource.java      |  2 +-
 .../apache/hadoop/hbase/wal/FaultyFSLog.java    |  4 +-
 .../apache/hadoop/hbase/wal/IOTestProvider.java |  4 +-
 .../apache/hadoop/hbase/wal/TestWALFactory.java |  2 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |  6 +--
 21 files changed, 133 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index e95a392..af20f40 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -221,6 +221,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
   public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
 
+  public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync";
+  public static final boolean DEFAULT_WAL_HSYNC = false;
+
   /**
    * Longest time we'll wait on a sequenceid.
    * Sequenceid comes up out of the WAL subsystem. WAL subsystem can go bad or a test might use
@@ -786,9 +789,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
      */
     this.rowProcessorTimeout = conf.getLong(
         "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
-    this.durability = htd.getDurability() == Durability.USE_DEFAULT
-        ? DEFAULT_DURABILITY
-        : htd.getDurability();
+
+    boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC);
+    /**
+     * This is the global default value for durability. All tables/mutations not defining a
+     * durability or using USE_DEFAULT will default to this value.
+     */
+    Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL;
+    this.durability =
+        htd.getDurability() == Durability.USE_DEFAULT ? defaultDurability : htd.getDurability();
+
     if (rsServices != null) {
       this.rsAccounting = this.rsServices.getRegionServerAccounting();
       // don't initialize coprocessors if not running within a regionserver
@@ -8758,9 +8768,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // nothing do to
         break;
       case SYNC_WAL:
+        this.wal.sync(txid, false);
+        break;
       case FSYNC_WAL:
         // sync the WAL edit (SYNC and FSYNC treated the same for now)
-        this.wal.sync(txid);
+        this.wal.sync(txid, true);
         break;
       default:
         throw new RuntimeException("Unknown durability " + durability);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 593132f..9cef728 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
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -279,6 +280,8 @@ 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 boolean useHsync;
+
   private final int slowSyncNs;
 
   private final long walSyncTimeout;
@@ -534,6 +537,8 @@ public class FSHLog implements WAL {
       ", prefix=" + this.logFilePrefix + ", suffix=" + logFileSuffix + ", logDir=" +
       this.fullPathLogDir + ", archiveDir=" + this.fullPathArchiveDir);
 
+    this.useHsync = conf.getBoolean(HRegion.WAL_HSYNC_CONF_KEY, HRegion.DEFAULT_WAL_HSYNC);
+
     // rollWriter sets this.hdfs_out if it can.
     rollWriter();
 
@@ -673,7 +678,7 @@ public class FSHLog implements WAL {
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
     try {
-      nextWriter.sync();
+      nextWriter.sync(useHsync);
       postSync(System.nanoTime() - startTimeNanos, 0);
     } catch (IOException e) {
       // optimization failed, no need to abort here.
@@ -1280,7 +1285,7 @@ public class FSHLog implements WAL {
           Throwable lastException = null;
           try {
             Trace.addTimelineAnnotation("syncing writer");
-            writer.sync();
+            writer.sync(takeSyncFuture.isForceSync());
             Trace.addTimelineAnnotation("writer synced");
             currentSequence = updateHighestSyncedSequence(currentSequence);
           } catch (IOException e) {
@@ -1383,20 +1388,20 @@ public class FSHLog implements WAL {
   }
 
   private SyncFuture publishSyncOnRingBuffer(long sequence) {
-    return publishSyncOnRingBuffer(sequence, null);
+    return publishSyncOnRingBuffer(sequence, null, false);
   }
 
   private long getSequenceOnRingBuffer() {
     return this.disruptor.getRingBuffer().next();
   }
 
-  private SyncFuture publishSyncOnRingBuffer(Span span) {
+  private SyncFuture publishSyncOnRingBuffer(Span span, boolean forceSync) {
     long sequence = this.disruptor.getRingBuffer().next();
-    return publishSyncOnRingBuffer(sequence, span);
+    return publishSyncOnRingBuffer(sequence, span, forceSync);
   }
 
-  private SyncFuture publishSyncOnRingBuffer(long sequence, Span span) {
-    SyncFuture syncFuture = getSyncFuture(sequence, span);
+  private SyncFuture publishSyncOnRingBuffer(long sequence, Span span, boolean forceSync) {
+    SyncFuture syncFuture = getSyncFuture(sequence, span).setForceSync(forceSync);
     try {
       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
       truck.loadPayload(syncFuture);
@@ -1407,8 +1412,8 @@ public class FSHLog implements WAL {
   }
 
   // Sync all known transactions
-  private Span publishSyncThenBlockOnCompletion(Span span) throws IOException {
-    return blockOnSync(publishSyncOnRingBuffer(span));
+  private Span publishSyncThenBlockOnCompletion(Span span, boolean forceSync) throws IOException {
+    return blockOnSync(publishSyncOnRingBuffer(span, forceSync));
   }
 
   private Span blockOnSync(final SyncFuture syncFuture) throws IOException {
@@ -1503,9 +1508,14 @@ public class FSHLog implements WAL {
 
   @Override
   public void sync() throws IOException {
+    sync(useHsync);
+  }
+
+  @Override
+  public void sync(boolean forceSync) throws IOException {
     TraceScope scope = Trace.startSpan("FSHLog.sync");
     try {
-      scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
+      scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach(), forceSync));
     } finally {
       assert scope == NullScope.INSTANCE || !scope.isDetached();
       scope.close();
@@ -1514,13 +1524,18 @@ public class FSHLog implements WAL {
 
   @Override
   public void sync(long txid) throws IOException {
-    if (this.highestSyncedSequence.get() >= txid){
+    sync(txid, useHsync);
+  }
+
+  @Override
+  public void sync(long txid, boolean forceSync) throws IOException {
+    if (this.highestSyncedSequence.get() >= txid) {
       // Already sync'd.
       return;
     }
     TraceScope scope = Trace.startSpan("FSHLog.sync");
     try {
-      scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
+      scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach(), forceSync));
     } finally {
       assert scope == NullScope.INSTANCE || !scope.isDetached();
       scope.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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..cb9e5a5 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
@@ -164,11 +164,15 @@ public class ProtobufLogWriter extends WriterBase {
   }
 
   @Override
-  public void sync() throws IOException {
+  public void sync(boolean forceSync) throws IOException {
     FSDataOutputStream fsdos = this.output;
     if (fsdos == null) return; // Presume closed
     fsdos.flush();
-    fsdos.hflush();
+    if (forceSync) {
+      fsdos.hsync();
+    } else {
+      fsdos.hflush();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 0720de8..c9e893c 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
@@ -80,6 +80,8 @@ class SyncFuture {
    */
   private Span span;
 
+  private boolean forceSync;
+
   /**
    * 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.
@@ -120,6 +122,15 @@ class SyncFuture {
     return this.ringBufferSequence;
   }
 
+  synchronized boolean isForceSync() {
+    return forceSync;
+  }
+
+  synchronized SyncFuture setForceSync(boolean forceSync) {
+    this.forceSync = forceSync;
+    return this;
+  }
+
   /**
    * Retrieve the {@code span} instance from this Future. EventHandler calls
    * this method to continue the span. Thread waiting on this Future musn't call

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 3dc1d5a..8931c08 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
@@ -199,6 +199,16 @@ class DisabledWALProvider implements WALProvider {
     }
 
     @Override
+    public void sync(boolean forceSync) throws IOException {
+      sync();
+    }
+
+    @Override
+    public void sync(long txid, boolean forceSync) throws IOException {
+      sync(txid);
+    }
+
+    @Override
     public Long startCacheFlush(final byte[] encodedRegionName, Set<byte[]> flushedFamilyNames) {
       if (closed.get()) return null;
       return HConstants.NO_SEQNUM;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 cc2f42a..1b053b2 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
@@ -137,6 +137,21 @@ public interface WAL extends Closeable {
   void sync(long txid) throws IOException;
 
   /**
+   * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
+   *          vs hsync.
+   * @throws IOException
+   */
+  void sync(boolean forceSync) throws IOException;
+
+  /**
+   * @param txid
+   * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop hflush
+   *          vs hsync.
+   * @throws IOException
+   */
+  void sync(long txid, boolean forceSync) throws IOException;
+
+  /**
    * WAL keeps track of the sequence numbers that are as yet not flushed im memstores
    * in order to be able to do accounting to figure which WALs can be let go. This method tells WAL
    * that some region is about to flush. The flush can be the whole region or for a column family

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 57ffd6b..8751c1f 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
@@ -80,7 +80,12 @@ public interface WALProvider {
   // Writers are used internally. Users outside of the WAL should be relying on the
   // interface provided by WAL.
   interface Writer extends Closeable {
-    void sync() throws IOException;
+    /**
+     * @param forceSync Flag to force sync rather than flushing to the buffer. Example - Hadoop
+     *          hflush vs hsync.
+     * @throws IOException
+     */
+    void sync(boolean forceSync) throws IOException;
     void append(WAL.Entry entry) throws IOException;
     long getLength() throws IOException;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRollbackFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRollbackFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRollbackFromClient.java
index 9230f31..f91adf4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRollbackFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRollbackFromClient.java
@@ -312,10 +312,20 @@ public class TestRollbackFromClient {
 
     @Override
     public void sync(long txid) throws IOException {
+      sync(txid, false);
+    }
+
+    @Override
+    public void sync(boolean forceSync) throws IOException {
+      delegation.sync(forceSync);
+    }
+
+    @Override
+    public void sync(long txid, boolean forceSync) throws IOException {
       if (SHOULD_FAIL.get()) {
         throw new IOException("[TESTING] we need the failure!!!");
       }
-      delegation.sync(txid);
+      delegation.sync(txid, forceSync);
     }
 
     @Override
@@ -347,6 +357,5 @@ public class TestRollbackFromClient {
     public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) {
       return delegation.getEarliestMemstoreSeqNum(encodedRegionName, familyName);
     }
-
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 945361f..1c32b06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -133,13 +133,13 @@ public class TestFailedAppendAndSync {
               w.close();
             }
 
-            @Override
-            public void sync() throws IOException {
-              if (throwSyncException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.sync();
+          @Override
+          public void sync(boolean forceSync) throws IOException {
+            if (throwSyncException) {
+              throw new IOException("FAKE! Failed to replace a bad datanode...");
             }
+            w.sync(forceSync);
+          }
 
             @Override
             public void append(Entry entry) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index e93fcf0..9c3b7af8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -325,9 +325,9 @@ public class TestHRegion {
       }
 
       @Override
-      public void sync(long txid) throws IOException {
+      public void sync(long txid, boolean forceSync) throws IOException {
         storeFlushCtx.prepare();
-        super.sync(txid);
+        super.sync(txid, forceSync);
       }
     }
 
@@ -1170,8 +1170,8 @@ public class TestHRegion {
           }
 
           @Override
-          public void sync() throws IOException {
-            w.sync();
+          public void sync(boolean forceSync) throws IOException {
+            w.sync(forceSync);
           }
 
           @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 0a0393a..046784a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -172,11 +172,11 @@ public class TestWALLockup {
           }
 
           @Override
-          public void sync() throws IOException {
+          public void sync(boolean forceSync) throws IOException {
             if (throwException) {
               throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
             }
-            w.sync();
+            w.sync(forceSync);
           }
 
           @Override
@@ -327,12 +327,11 @@ public class TestWALLockup {
           }
 
           @Override
-          public void sync() throws IOException {
+          public void sync(boolean forceSync) throws IOException {
             if (throwException) {
-              throw new IOException(
-                  "FAKE! Failed to replace a bad datanode...SYNC");
+              throw new IOException("FAKE! Failed to replace a bad datanode...SYNC");
             }
-            w.sync();
+            w.sync(forceSync);
           }
 
           @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 6ebafc9..7aff0e5 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
@@ -193,7 +193,7 @@ public class SequenceFileLogWriter extends WriterBase {
   }
 
   @Override
-  public void sync() throws IOException {
+  public void sync(boolean forceSync) throws IOException {
     try {
       this.writer.syncFs();
     } catch (NullPointerException npe) {
@@ -219,4 +219,5 @@ public class SequenceFileLogWriter extends WriterBase {
   public FSDataOutputStream getWriterFSDataOutputStream() {
     return this.writer_out;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 bca4a7e..f8898c7 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
@@ -56,9 +56,9 @@ public class TestLogRollingNoCluster {
   /** ProtobufLogWriter that simulates higher latencies in sync() call */
   public static class HighLatencySyncWriter extends  ProtobufLogWriter {
     @Override
-    public void sync() throws IOException {
+    public void sync(boolean forceSync) throws IOException {
       Threads.sleep(ThreadLocalRandom.current().nextInt(10));
-      super.sync();
+      super.sync(forceSync);
       Threads.sleep(ThreadLocalRandom.current().nextInt(10));
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 3d10a39..c8404c4 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
@@ -170,7 +170,7 @@ public class TestProtobufLog {
         }
         writer.append(new WAL.Entry(key, edit));
       }
-      writer.sync();
+      writer.sync(false);
       if (withTrailer) writer.close();
 
       // Now read the log using standard means.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 1138f65..5bdfa3e 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
@@ -120,7 +120,7 @@ public class TestReadOldRootAndMetaEdits {
       writer.append(oldMetaEntry);
 
       // sync/close the writer
-      writer.sync();
+      writer.sync(false);
       writer.close();
 
       // read the log and see things are okay.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 938520e..de71f8f 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
@@ -1253,7 +1253,7 @@ public class TestWALReplay {
     for (FSWALEntry entry : entries) {
       writer.append(entry);
     }
-    writer.sync();
+    writer.sync(false);
     writer.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/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 6e6fe9a..5d9059a 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
@@ -120,7 +120,7 @@ public class TestReplicationSource {
       WALKey key = new WALKey(b, TableName.valueOf(b), 0, 0,
           HConstants.DEFAULT_CLUSTER_ID);
       writer.append(new WAL.Entry(key, edit));
-      writer.sync();
+      writer.sync(false);
     }
     writer.close();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index 2954096..93c0aea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
@@ -57,11 +57,11 @@ public class FaultyFSLog extends FSHLog {
   }
   
   @Override
-  public void sync(long txid) throws IOException {
+  public void sync(long txid, boolean forceSync) throws IOException {
     if (this.ft == FailureType.SYNC) {
       throw new IOException("sync");
     }
-    super.sync(txid);
+    super.sync(txid, forceSync);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index a044bcf..97fa2c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -232,9 +232,9 @@ public class IOTestProvider implements WALProvider {
     }
 
     @Override
-    public void sync() throws IOException {
+    public void sync(boolean forceSync) throws IOException {
       if (doSyncs) {
-        super.sync();
+        super.sync(forceSync);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index 621d092..42ef1a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -677,7 +677,7 @@ public class TestWALFactory {
         }
         sflw.append(new WAL.Entry(key, edit));
       }
-      sflw.sync();
+      sflw.sync(false);
       sflw.close();
 
       // Now read the log using standard means.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f976b3a8/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 3ae155c..92fe4db 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -1353,7 +1353,7 @@ public class TestWALSplit {
     WALKey key = new WALKey(hri.getEncodedNameAsBytes(), TABLE_NAME, 1,
         EnvironmentEdgeManager.currentTime(), HConstants.DEFAULT_CLUSTER_ID);
     w.append(new Entry(key, edit));
-    w.sync();
+    w.sync(false);
   }
 
   private static void appendRegionEvent(Writer w, String region) throws IOException {
@@ -1371,7 +1371,7 @@ public class TestWALSplit {
         HConstants.DEFAULT_CLUSTER_ID);
     w.append(
         new Entry(walKey, new WALEdit().add(kv)));
-    w.sync();
+    w.sync(false);
   }
 
   public static long appendEntry(Writer writer, TableName table, byte[] region,
@@ -1381,7 +1381,7 @@ public class TestWALSplit {
     LOG.info(Thread.currentThread().getName() + " append");
     writer.append(createTestEntry(table, region, row, family, qualifier, value, seq));
     LOG.info(Thread.currentThread().getName() + " sync");
-    writer.sync();
+    writer.sync(false);
     return seq;
   }