You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/02/27 00:09:22 UTC

[01/37] hbase git commit: HBASE-15247 InclusiveStopFilter does not respect reverse Filter property. (Amal Joshy)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 2966eee60 -> 8f6e29785


HBASE-15247 InclusiveStopFilter does not respect reverse Filter property. (Amal Joshy)


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

Branch: refs/heads/hbase-12439
Commit: 2d66cd86d08120887efe05b65fa53bd667a5be76
Parents: 2966eee
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Feb 23 09:21:37 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Tue Feb 23 09:21:37 2016 +0530

----------------------------------------------------------------------
 .../hbase/filter/InclusiveStopFilter.java       |  4 +--
 .../apache/hadoop/hbase/filter/TestFilter.java  | 37 ++++++++++++++++++++
 2 files changed, 38 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2d66cd86/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
index 3412c5c..1096f5e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
@@ -62,9 +62,7 @@ public class InclusiveStopFilter extends FilterBase {
   public boolean filterRowKey(Cell firstRowCell) {
     // if stopRowKey is <= buffer, then true, filter row.
     int cmp = CellComparator.COMPARATOR.compareRows(firstRowCell, stopRowKey, 0, stopRowKey.length);
-    if (cmp > 0) {
-      done = true;
-    }
+    done = reversed ? cmp < 0 : cmp > 0;
     return done;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d66cd86/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index 1e89685..1dd6616 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -717,6 +717,43 @@ public class TestFilter {
   }
 
   @Test
+  public void testInclusiveStopFilterWithReverseScan() throws IOException {
+
+    // Grab rows from group one
+
+    // If we just use start/stop row, we get total/2 - 1 rows
+    long expectedRows = (this.numRows / 2) - 1;
+    long expectedKeys = this.colsPerRow;
+    Scan s = new Scan(Bytes.toBytes("testRowOne-3"), Bytes.toBytes("testRowOne-0"));
+    s.setReversed(true);
+    verifyScan(s, expectedRows, expectedKeys);
+
+    // Now use start row with inclusive stop filter
+    expectedRows = this.numRows / 2;
+    s = new Scan(Bytes.toBytes("testRowOne-3"));
+    s.setReversed(true);
+    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-0")));
+    verifyScan(s, expectedRows, expectedKeys);
+
+    // Grab rows from group two
+
+    // If we just use start/stop row, we get total/2 - 1 rows
+    expectedRows = (this.numRows / 2) - 1;
+    expectedKeys = this.colsPerRow;
+    s = new Scan(Bytes.toBytes("testRowTwo-3"), Bytes.toBytes("testRowTwo-0"));
+    s.setReversed(true);
+    verifyScan(s, expectedRows, expectedKeys);
+
+    // Now use start row with inclusive stop filter
+    expectedRows = this.numRows / 2;
+    s = new Scan(Bytes.toBytes("testRowTwo-3"));
+    s.setReversed(true);
+    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-0")));
+    verifyScan(s, expectedRows, expectedKeys);
+
+  }
+
+  @Test
   public void testQualifierFilter() throws IOException {
 
     // Match two keys (one from each family) in half the rows


[21/37] hbase git commit: HBASE-15144 Procedure v2 - Web UI displaying Store state

Posted by sy...@apache.org.
HBASE-15144 Procedure v2 - Web UI displaying Store state


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

Branch: refs/heads/hbase-12439
Commit: 40c55915e7a45a639adb7f7a370a04f38058ac26
Parents: 77133fd
Author: Samir Ahmic <sa...@personal.com>
Authored: Wed Feb 24 16:05:24 2016 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Feb 25 10:46:56 2016 -0800

----------------------------------------------------------------------
 .../procedure2/store/wal/ProcedureWALFile.java  |  32 +++--
 .../store/wal/ProcedureWALFormat.java           |   9 +-
 .../store/wal/ProcedureWALFormatReader.java     |   8 +-
 .../procedure2/store/wal/WALProcedureStore.java | 108 +++++++++++++----
 .../org/apache/hadoop/hbase/master/HMaster.java |   8 +-
 .../hbase-webapps/master/procedures.jsp         | 118 ++++++++++++++++++-
 6 files changed, 244 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40c55915/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index 6493526..097cd29 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -22,12 +22,12 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
@@ -42,24 +42,29 @@ public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
 
   private ProcedureWALHeader header;
   private FSDataInputStream stream;
-  private FileStatus logStatus;
   private FileSystem fs;
   private Path logFile;
   private long startPos;
   private long minProcId;
   private long maxProcId;
+  private long logSize;
+  private long timestamp;
 
   public ProcedureWALFile(final FileSystem fs, final FileStatus logStatus) {
     this.fs = fs;
-    this.logStatus = logStatus;
     this.logFile = logStatus.getPath();
+    this.logSize = logStatus.getLen();
+    this.timestamp = logStatus.getModificationTime();
   }
 
-  public ProcedureWALFile(FileSystem fs, Path logFile, ProcedureWALHeader header, long startPos) {
+  public ProcedureWALFile(FileSystem fs, Path logFile, ProcedureWALHeader header,
+      long startPos, long timestamp) {
     this.fs = fs;
-    this.logFile = logFile;
     this.header = header;
+    this.logFile = logFile;
     this.startPos = startPos;
+    this.logSize = startPos;
+    this.timestamp = timestamp;
   }
 
   public void open() throws IOException {
@@ -77,7 +82,7 @@ public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
 
   public ProcedureWALTrailer readTrailer() throws IOException {
     try {
-      return ProcedureWALFormat.readTrailer(stream, startPos, logStatus.getLen());
+      return ProcedureWALFormat.readTrailer(stream, startPos, logSize);
     } finally {
       stream.seek(startPos);
     }
@@ -112,6 +117,10 @@ public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
     return header;
   }
 
+  public long getTimestamp() {
+    return timestamp;
+  }
+
   public boolean isCompacted() {
     return header.getType() == ProcedureWALFormat.LOG_TYPE_COMPACTED;
   }
@@ -121,7 +130,14 @@ public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
   }
 
   public long getSize() {
-    return logStatus != null ? logStatus.getLen() : 0;
+    return logSize;
+  }
+
+  /**
+   * Used to update in-progress log sizes. the FileStatus will report 0 otherwise.
+   */
+  void addToSize(long size) {
+    this.logSize += size;
   }
 
   public void removeFile() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/40c55915/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
index 2b393c0..add7d03 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
@@ -23,14 +23,14 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Iterator;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureLoader;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
 import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
@@ -113,7 +113,7 @@ public final class ProcedureWALFormat {
    * |      offset     |-----+
    * +-----------------+
    */
-  public static void writeTrailer(FSDataOutputStream stream, ProcedureStoreTracker tracker)
+  public static long writeTrailer(FSDataOutputStream stream, ProcedureStoreTracker tracker)
       throws IOException {
     long offset = stream.getPos();
 
@@ -128,6 +128,7 @@ public final class ProcedureWALFormat {
     stream.write(TRAILER_VERSION);
     StreamUtils.writeLong(stream, TRAILER_MAGIC);
     StreamUtils.writeLong(stream, offset);
+    return stream.getPos() - offset;
   }
 
   public static ProcedureWALHeader readHeader(InputStream stream)

http://git-wip-us.apache.org/repos/asf/hbase/blob/40c55915/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
index 73bd7bc..312eedb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
@@ -23,15 +23,17 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
 
+import com.google.protobuf.InvalidProtocolBufferException;
+
 /**
  * Helper class that loads the procedures stored in a WAL
  */
@@ -142,7 +144,7 @@ public class ProcedureWALFormatReader {
             throw new CorruptedWALProcedureStoreException("Invalid entry: " + entry);
         }
       }
-    } catch (IOException e) {
+    } catch (InvalidProtocolBufferException e) {
       LOG.error("got an exception while reading the procedure WAL: " + log, e);
       loader.markCorruptedWAL(log, e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40c55915/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 280c6ca..5ac421c 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -18,23 +18,24 @@
 
 package org.apache.hadoop.hbase.procedure2.store.wal;
 
-import java.io.IOException;
 import java.io.FileNotFoundException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.concurrent.LinkedTransferQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.Arrays;
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Set;
+import java.util.concurrent.LinkedTransferQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.collections.buffer.CircularFifoBuffer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -98,6 +99,10 @@ public class WALProcedureStore extends ProcedureStoreBase {
   private static final String ROLL_THRESHOLD_CONF_KEY = "hbase.procedure.store.wal.roll.threshold";
   private static final long DEFAULT_ROLL_THRESHOLD = 32 * 1024 * 1024; // 32M
 
+  private static final String STORE_WAL_SYNC_STATS_COUNT =
+      "hbase.procedure.store.wal.sync.stats.count";
+  private static final int DEFAULT_SYNC_STATS_COUNT = 10;
+
   private final LinkedList<ProcedureWALFile> logs = new LinkedList<ProcedureWALFile>();
   private final ProcedureStoreTracker storeTracker = new ProcedureStoreTracker();
   private final ReentrantLock lock = new ReentrantLock();
@@ -133,6 +138,37 @@ public class WALProcedureStore extends ProcedureStoreBase {
   private boolean useHsync;
   private int syncWaitMsec;
 
+  // Variables used for UI display
+  private CircularFifoBuffer syncMetricsBuffer;
+
+  public static class SyncMetrics {
+    private long timestamp;
+    private long syncWaitMs;
+    private long totalSyncedBytes;
+    private int syncedEntries;
+    private float syncedPerSec;
+
+    public long getTimestamp() {
+      return timestamp;
+    }
+
+    public long getSyncWaitMs() {
+      return syncWaitMs;
+    }
+
+    public long getTotalSyncedBytes() {
+      return totalSyncedBytes;
+    }
+
+    public long getSyncedEntries() {
+      return syncedEntries;
+    }
+
+    public float getSyncedPerSec() {
+      return syncedPerSec;
+    }
+  }
+
   public WALProcedureStore(final Configuration conf, final FileSystem fs, final Path logDir,
       final LeaseRecovery leaseRecovery) {
     this.fs = fs;
@@ -166,6 +202,10 @@ public class WALProcedureStore extends ProcedureStoreBase {
     syncWaitMsec = conf.getInt(SYNC_WAIT_MSEC_CONF_KEY, DEFAULT_SYNC_WAIT_MSEC);
     useHsync = conf.getBoolean(USE_HSYNC_CONF_KEY, DEFAULT_USE_HSYNC);
 
+    // WebUI
+    syncMetricsBuffer = new CircularFifoBuffer(
+      conf.getInt(STORE_WAL_SYNC_STATS_COUNT, DEFAULT_SYNC_STATS_COUNT));
+
     // Init sync thread
     syncThread = new Thread("WALProcedureStoreSyncThread") {
       @Override
@@ -509,6 +549,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
   }
 
   private void syncLoop() throws Throwable {
+    long totalSyncedToStore = 0;
     inSync.set(false);
     lock.lock();
     try {
@@ -533,23 +574,37 @@ public class WALProcedureStore extends ProcedureStoreBase {
               continue;
             }
           }
-
           // Wait SYNC_WAIT_MSEC or the signal of "slots full" before flushing
-          long syncWaitSt = System.currentTimeMillis();
+          final long syncWaitSt = System.currentTimeMillis();
           if (slotIndex != slots.length) {
             slotCond.await(syncWaitMsec, TimeUnit.MILLISECONDS);
           }
-          long syncWaitMs = System.currentTimeMillis() - syncWaitSt;
+
+          final long currentTs = System.currentTimeMillis();
+          final long syncWaitMs = currentTs - syncWaitSt;
+          final float rollSec = getMillisFromLastRoll() / 1000.0f;
+          final float syncedPerSec = totalSyncedToStore / rollSec;
           if (LOG.isTraceEnabled() && (syncWaitMs > 10 || slotIndex < slots.length)) {
-            float rollSec = getMillisFromLastRoll() / 1000.0f;
             LOG.trace(String.format("Sync wait %s, slotIndex=%s , totalSynced=%s (%s/sec)",
                       StringUtils.humanTimeDiff(syncWaitMs), slotIndex,
-                      StringUtils.humanSize(totalSynced.get()),
-                      StringUtils.humanSize(totalSynced.get() / rollSec)));
+                      StringUtils.humanSize(totalSyncedToStore),
+                      StringUtils.humanSize(syncedPerSec)));
           }
 
+          // update webui circular buffers (TODO: get rid of allocations)
+          final SyncMetrics syncMetrics = new SyncMetrics();
+          syncMetrics.timestamp = currentTs;
+          syncMetrics.syncWaitMs = syncWaitMs;
+          syncMetrics.syncedEntries = slotIndex;
+          syncMetrics.totalSyncedBytes = totalSyncedToStore;
+          syncMetrics.syncedPerSec = syncedPerSec;
+          syncMetricsBuffer.add(syncMetrics);
+
+          // sync
           inSync.set(true);
-          totalSynced.addAndGet(syncSlots());
+          long slotSize = syncSlots();
+          logs.getLast().addToSize(slotSize);
+          totalSyncedToStore = totalSynced.addAndGet(slotSize);
           slotIndex = 0;
           inSync.set(false);
         } catch (InterruptedException e) {
@@ -569,6 +624,15 @@ public class WALProcedureStore extends ProcedureStoreBase {
     }
   }
 
+  public ArrayList<SyncMetrics> getSyncMetrics() {
+    lock.lock();
+    try {
+      return new ArrayList<SyncMetrics>(syncMetricsBuffer);
+    } finally {
+      lock.unlock();
+    }
+  }
+
   private long syncSlots() throws Throwable {
     int retry = 0;
     int logRolled = 0;
@@ -647,14 +711,14 @@ public class WALProcedureStore extends ProcedureStoreBase {
     }
   }
 
-  private long getMillisToNextPeriodicRoll() {
+  public long getMillisToNextPeriodicRoll() {
     if (lastRollTs.get() > 0 && periodicRollMsec > 0) {
       return periodicRollMsec - getMillisFromLastRoll();
     }
     return Long.MAX_VALUE;
   }
 
-  private long getMillisFromLastRoll() {
+  public long getMillisFromLastRoll() {
     return (System.currentTimeMillis() - lastRollTs.get());
   }
 
@@ -761,8 +825,9 @@ public class WALProcedureStore extends ProcedureStoreBase {
     stream = newStream;
     flushLogId = logId;
     totalSynced.set(0);
-    lastRollTs.set(System.currentTimeMillis());
-    logs.add(new ProcedureWALFile(fs, newLogFile, header, startPos));
+    long rollTs = System.currentTimeMillis();
+    lastRollTs.set(rollTs);
+    logs.add(new ProcedureWALFile(fs, newLogFile, header, startPos, rollTs));
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Roll new state log: " + logId);
@@ -776,7 +841,8 @@ public class WALProcedureStore extends ProcedureStoreBase {
         try {
           ProcedureWALFile log = logs.getLast();
           log.setProcIds(storeTracker.getUpdatedMinProcId(), storeTracker.getUpdatedMaxProcId());
-          ProcedureWALFormat.writeTrailer(stream, storeTracker);
+          long trailerSize = ProcedureWALFormat.writeTrailer(stream, storeTracker);
+          log.addToSize(trailerSize);
         } catch (IOException e) {
           LOG.warn("Unable to write the trailer: " + e.getMessage());
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40c55915/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 3cf750e..5d8c325 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -336,7 +336,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   // handle table states
   private TableStateManager tableStateManager;
-  
+
   private long splitPlanCount;
   private long mergePlanCount;
 
@@ -2159,6 +2159,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     return procedureStore != null ? procedureStore.getActiveLogs().size() : 0;
   }
 
+  public WALProcedureStore getWalProcedureStore() {
+    return procedureStore;
+  }
+
   public int getRegionServerInfoPort(final ServerName sn) {
     RegionServerInfo info = this.regionServerTracker.getRegionServerInfo(sn);
     if (info == null || info.getInfoPort() == 0) {
@@ -2358,7 +2362,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     return regionStates.getAverageLoad();
   }
-  
+
   /*
    * @return the count of region split plans executed
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/40c55915/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
index 443d9d6..eb5ea39 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/procedures.jsp
@@ -21,18 +21,31 @@
   import="static org.apache.commons.lang.StringEscapeUtils.escapeXml"
   import="java.util.Collections"
   import="java.util.Comparator"
+  import="java.util.ArrayList"
   import="java.util.Date"
   import="java.util.List"
+  import="java.util.Set"
+  import="org.apache.hadoop.conf.Configuration"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
   import="org.apache.hadoop.hbase.ProcedureInfo"
   import="org.apache.hadoop.hbase.master.HMaster"
   import="org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv"
   import="org.apache.hadoop.hbase.procedure2.ProcedureExecutor"
+  import="org.apache.hadoop.hbase.procedure2.store.wal.ProcedureWALFile"
+  import="org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore"
+  import="org.apache.hadoop.hbase.procedure2.util.StringUtils"
+
 %>
 <%
   HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
   ProcedureExecutor<MasterProcedureEnv> procExecutor = master.getMasterProcedureExecutor();
+  WALProcedureStore walStore = master.getWalProcedureStore();
 
+  ArrayList<WALProcedureStore.SyncMetrics> syncMetricsBuff = walStore.getSyncMetrics();
+  long millisToNextRoll = walStore.getMillisToNextPeriodicRoll();
+  long millisFromLastRoll = walStore.getMillisFromLastRoll();
+  ArrayList<ProcedureWALFile> procedureWALFiles = walStore.getActiveLogs();
+  Set<ProcedureWALFile> corruptedWALFiles = walStore.getCorruptedLogs();
   List<ProcedureInfo> procedures = procExecutor.listProcedures();
   Collections.sort(procedures, new Comparator<ProcedureInfo>() {
     @Override
@@ -118,7 +131,110 @@
     <% } %>
   </table>
 </div>
-
+<br>
+<div class="container-fluid content">
+  <div class="row">
+    <div class="page-header">
+      <h2>Procedure WAL State</h2>
+    </div>
+  </div>
+<div class="tabbable">
+  <ul class="nav nav-pills">
+    <li class="active">
+      <a href="#tab_WALFiles" data-toggle="tab">WAL files</a>
+    </li>
+    <li class="">
+      <a href="#tab_WALFilesCorrupted" data-toggle="tab">Corrupted WAL files</a>
+     </li>
+    <li class="">
+      <a href="#tab_WALRollTime" data-toggle="tab">WAL roll time</a>
+     </li>
+     <li class="">
+       <a href="#tab_SyncStats" data-toggle="tab">Sync stats</a>
+     </li>
+  </ul>
+    <div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
+      <div class="tab-pane active" id="tab_WALFiles">
+        <% if (procedureWALFiles != null && procedureWALFiles.size() > 0) { %>
+          <table class="table table-striped">
+            <tr>
+              <th>LogID</th>
+              <th>Size</th>
+              <th>Timestamp</th>
+              <th>Path</th>
+            </tr>
+            <% for (int i = procedureWALFiles.size() - 1; i >= 0; --i) { %>
+            <%    ProcedureWALFile pwf = procedureWALFiles.get(i); %>
+            <tr>
+              <td> <%= pwf.getLogId() %></td>
+              <td> <%= StringUtils.humanSize(pwf.getSize()) %> </td>
+              <td> <%= new Date(pwf.getTimestamp()) %></a></td>
+              <td> <%= escapeXml(pwf.toString()) %></t>
+            </tr>
+            <% } %>
+          </table>
+        <% } else {%>
+          <p> No WAL files</p>
+        <% } %>
+      </div>
+      <div class="tab-pane" id="tab_WALFilesCorrupted">
+      <% if (corruptedWALFiles != null && corruptedWALFiles.size() > 0) { %>
+        <table class="table table-striped">
+          <tr>
+            <th>LogID</th>
+            <th>Size</th>
+            <th>Timestamp</th>
+            <th>Path</th>
+          </tr>
+          <% for (ProcedureWALFile cwf:corruptedWALFiles) { %>
+          <tr>
+            <td> <%= cwf.getLogId() %></td>
+            <td> <%= StringUtils.humanSize(cwf.getSize()) %> </td>
+            <td> <%= new Date(cwf.getTimestamp()) %></a></td>
+            <td> <%= escapeXml(cwf.toString()) %></t>
+          </tr>
+          <% } %>
+          </table>
+      <% } else {%>
+        <p> No corrupted WAL files</p>
+      <% } %>
+      </div>
+      <div class="tab-pane" id="tab_WALRollTime">
+        <table class="table table-striped">
+          <tr>
+            <th> Milliseconds to next roll</th>
+            <th> Milliseconds from last roll</th>
+          </tr>
+          <tr>
+            <td> <%=StringUtils.humanTimeDiff(millisToNextRoll)  %></td>
+            <td> <%=StringUtils.humanTimeDiff(millisFromLastRoll) %></td>
+          </tr>
+        </table>
+      </div>
+      <div class="tab-pane" id="tab_SyncStats">
+        <table class="table table-striped">
+          <tr>
+            <th> Time</th>
+            <th> Sync Wait</th>
+            <th> Last num of synced entries</th>
+            <th> Total Synced</th>
+            <th> Synced per second</th>
+          </tr>
+          <% for (int i = syncMetricsBuff.size() - 1; i >= 0; --i) { %>
+          <%    WALProcedureStore.SyncMetrics syncMetrics = syncMetricsBuff.get(i); %>
+          <tr>
+            <td> <%= new Date(syncMetrics.getTimestamp()) %></a></td>
+            <td> <%= StringUtils.humanTimeDiff(syncMetrics.getSyncWaitMs()) %></td>
+            <td> <%= syncMetrics.getSyncedEntries() %></td>
+            <td> <%= StringUtils.humanSize(syncMetrics.getTotalSyncedBytes()) %></td>
+            <td> <%= StringUtils.humanSize(syncMetrics.getSyncedPerSec()) %></td>
+          </tr>
+          <%} %>
+        </table>
+        </div>
+      </div>
+  </div>
+</div>
 <script src="/static/js/jquery.min.js" type="text/javascript"></script>
 <script src="/static/js/bootstrap.min.js" type="text/javascript"></script>
 


[08/37] hbase git commit: HBASE-15277 TestRegionMergeTransactionOnCluster.testWholesomeMerge fails with no connection to master; ADDING DEBUGGING

Posted by sy...@apache.org.
HBASE-15277 TestRegionMergeTransactionOnCluster.testWholesomeMerge fails with no connection to master; ADDING DEBUGGING


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

Branch: refs/heads/hbase-12439
Commit: 28cd48b673ca743d193874b2951bc995699e8e89
Parents: 0024865
Author: stack <st...@apache.org>
Authored: Tue Feb 23 22:43:01 2016 -0800
Committer: stack <st...@apache.org>
Committed: Tue Feb 23 22:43:01 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/TableStateManager.java   | 2 +-
 .../regionserver/TestRegionMergeTransactionOnCluster.java   | 9 +++++++--
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/28cd48b6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index 12db91e..b6befaa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -131,7 +131,7 @@ public class TableStateManager {
       TableState.State tableState = getTableState(tableName);
       return TableState.isInStates(tableState, states);
     } catch (IOException e) {
-      LOG.error("Unable to get table " + tableName + " state, probably table not exists");
+      LOG.error("Unable to get table " + tableName + " state", e);
       return false;
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/28cd48b6/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index a532bb7..cd4410f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -78,6 +78,7 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 
 import com.google.common.base.Joiner;
@@ -94,6 +95,7 @@ import com.google.protobuf.ServiceException;
 public class TestRegionMergeTransactionOnCluster {
   private static final Log LOG = LogFactory
       .getLog(TestRegionMergeTransactionOnCluster.class);
+  @Rule public TestName name = new TestName();
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
       withLookingForStuckThread(true).build();
   private static final int NB_SERVERS = 3;
@@ -182,7 +184,6 @@ public class TestRegionMergeTransactionOnCluster {
    */
   @Test
   public void testMergeAndRestartingMaster() throws Exception {
-    LOG.info("Starting testMergeAndRestartingMaster");
     final TableName tableName = TableName.valueOf("testMergeAndRestartingMaster");
 
     // Create table and load data.
@@ -458,11 +459,15 @@ public class TestRegionMergeTransactionOnCluster {
     }
 
     Table table = TEST_UTIL.createTable(tablename, FAMILYNAME, splitRows);
+    LOG.info("Created " + table.getName());
     if (replication > 1) {
       HBaseTestingUtility.setReplicas(ADMIN, tablename, replication);
+      LOG.info("Set replication of " + replication + " on " + table.getName());
     }
     loadData(table);
+    LOG.info("Loaded " + table.getName());
     verifyRowCount(table, ROWSIZE);
+    LOG.info("Verified " + table.getName());
 
     // sleep here is an ugly hack to allow region transitions to finish
     long timeout = System.currentTimeMillis() + waitTime;
@@ -474,7 +479,7 @@ public class TestRegionMergeTransactionOnCluster {
         break;
       Thread.sleep(250);
     }
-
+    LOG.info("Getting regions of " + table.getName());
     tableRegions = MetaTableAccessor.getTableRegionsAndLocations(
         TEST_UTIL.getConnection(), tablename);
     LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));


[25/37] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 043d549..073eba9 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -8,6 +8,88 @@ public final class MasterProtos {
   public static void registerAllExtensions(
       com.google.protobuf.ExtensionRegistry registry) {
   }
+  /**
+   * Protobuf enum {@code hbase.pb.MasterSwitchType}
+   */
+  public enum MasterSwitchType
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>SPLIT = 0;</code>
+     */
+    SPLIT(0, 0),
+    /**
+     * <code>MERGE = 1;</code>
+     */
+    MERGE(1, 1),
+    ;
+
+    /**
+     * <code>SPLIT = 0;</code>
+     */
+    public static final int SPLIT_VALUE = 0;
+    /**
+     * <code>MERGE = 1;</code>
+     */
+    public static final int MERGE_VALUE = 1;
+
+
+    public final int getNumber() { return value; }
+
+    public static MasterSwitchType valueOf(int value) {
+      switch (value) {
+        case 0: return SPLIT;
+        case 1: return MERGE;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<MasterSwitchType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<MasterSwitchType>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<MasterSwitchType>() {
+            public MasterSwitchType findValueByNumber(int number) {
+              return MasterSwitchType.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.getDescriptor().getEnumTypes().get(0);
+    }
+
+    private static final MasterSwitchType[] VALUES = values();
+
+    public static MasterSwitchType valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private MasterSwitchType(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.MasterSwitchType)
+  }
+
   public interface AddColumnRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -28764,28 +28846,62 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.IsBalancerEnabledResponse)
   }
 
-  public interface NormalizeRequestOrBuilder
+  public interface SetSplitOrMergeEnabledRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // required bool enabled = 1;
+    /**
+     * <code>required bool enabled = 1;</code>
+     */
+    boolean hasEnabled();
+    /**
+     * <code>required bool enabled = 1;</code>
+     */
+    boolean getEnabled();
+
+    // optional bool synchronous = 2;
+    /**
+     * <code>optional bool synchronous = 2;</code>
+     */
+    boolean hasSynchronous();
+    /**
+     * <code>optional bool synchronous = 2;</code>
+     */
+    boolean getSynchronous();
+
+    // repeated .hbase.pb.MasterSwitchType switch_types = 3;
+    /**
+     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> getSwitchTypesList();
+    /**
+     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+     */
+    int getSwitchTypesCount();
+    /**
+     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchTypes(int index);
   }
   /**
-   * Protobuf type {@code hbase.pb.NormalizeRequest}
+   * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledRequest}
    */
-  public static final class NormalizeRequest extends
+  public static final class SetSplitOrMergeEnabledRequest extends
       com.google.protobuf.GeneratedMessage
-      implements NormalizeRequestOrBuilder {
-    // Use NormalizeRequest.newBuilder() to construct.
-    private NormalizeRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements SetSplitOrMergeEnabledRequestOrBuilder {
+    // Use SetSplitOrMergeEnabledRequest.newBuilder() to construct.
+    private SetSplitOrMergeEnabledRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private NormalizeRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private SetSplitOrMergeEnabledRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final NormalizeRequest defaultInstance;
-    public static NormalizeRequest getDefaultInstance() {
+    private static final SetSplitOrMergeEnabledRequest defaultInstance;
+    public static SetSplitOrMergeEnabledRequest getDefaultInstance() {
       return defaultInstance;
     }
 
-    public NormalizeRequest getDefaultInstanceForType() {
+    public SetSplitOrMergeEnabledRequest getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -28795,11 +28911,12 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private NormalizeRequest(
+    private SetSplitOrMergeEnabledRequest(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -28817,6 +28934,49 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              enabled_ = input.readBool();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              synchronous_ = input.readBool();
+              break;
+            }
+            case 24: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(3, rawValue);
+              } else {
+                if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                  switchTypes_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType>();
+                  mutable_bitField0_ |= 0x00000004;
+                }
+                switchTypes_.add(value);
+              }
+              break;
+            }
+            case 26: {
+              int length = input.readRawVarint32();
+              int oldLimit = input.pushLimit(length);
+              while(input.getBytesUntilLimit() > 0) {
+                int rawValue = input.readEnum();
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.valueOf(rawValue);
+                if (value == null) {
+                  unknownFields.mergeVarintField(3, rawValue);
+                } else {
+                  if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                    switchTypes_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType>();
+                    mutable_bitField0_ |= 0x00000004;
+                  }
+                  switchTypes_.add(value);
+                }
+              }
+              input.popLimit(oldLimit);
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -28825,44 +28985,109 @@ public final class MasterProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          switchTypes_ = java.util.Collections.unmodifiableList(switchTypes_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<NormalizeRequest> PARSER =
-        new com.google.protobuf.AbstractParser<NormalizeRequest>() {
-      public NormalizeRequest parsePartialFrom(
+    public static com.google.protobuf.Parser<SetSplitOrMergeEnabledRequest> PARSER =
+        new com.google.protobuf.AbstractParser<SetSplitOrMergeEnabledRequest>() {
+      public SetSplitOrMergeEnabledRequest parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new NormalizeRequest(input, extensionRegistry);
+        return new SetSplitOrMergeEnabledRequest(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<NormalizeRequest> getParserForType() {
+    public com.google.protobuf.Parser<SetSplitOrMergeEnabledRequest> getParserForType() {
       return PARSER;
     }
 
+    private int bitField0_;
+    // required bool enabled = 1;
+    public static final int ENABLED_FIELD_NUMBER = 1;
+    private boolean enabled_;
+    /**
+     * <code>required bool enabled = 1;</code>
+     */
+    public boolean hasEnabled() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bool enabled = 1;</code>
+     */
+    public boolean getEnabled() {
+      return enabled_;
+    }
+
+    // optional bool synchronous = 2;
+    public static final int SYNCHRONOUS_FIELD_NUMBER = 2;
+    private boolean synchronous_;
+    /**
+     * <code>optional bool synchronous = 2;</code>
+     */
+    public boolean hasSynchronous() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool synchronous = 2;</code>
+     */
+    public boolean getSynchronous() {
+      return synchronous_;
+    }
+
+    // repeated .hbase.pb.MasterSwitchType switch_types = 3;
+    public static final int SWITCH_TYPES_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> switchTypes_;
+    /**
+     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> getSwitchTypesList() {
+      return switchTypes_;
+    }
+    /**
+     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+     */
+    public int getSwitchTypesCount() {
+      return switchTypes_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchTypes(int index) {
+      return switchTypes_.get(index);
+    }
+
     private void initFields() {
+      enabled_ = false;
+      synchronous_ = false;
+      switchTypes_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
+      if (!hasEnabled()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -28870,6 +29095,15 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBool(1, enabled_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, synchronous_);
+      }
+      for (int i = 0; i < switchTypes_.size(); i++) {
+        output.writeEnum(3, switchTypes_.get(i).getNumber());
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -28879,6 +29113,23 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(1, enabled_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, synchronous_);
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < switchTypes_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeEnumSizeNoTag(switchTypes_.get(i).getNumber());
+        }
+        size += dataSize;
+        size += 1 * switchTypes_.size();
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -28896,12 +29147,24 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) obj;
 
       boolean result = true;
+      result = result && (hasEnabled() == other.hasEnabled());
+      if (hasEnabled()) {
+        result = result && (getEnabled()
+            == other.getEnabled());
+      }
+      result = result && (hasSynchronous() == other.hasSynchronous());
+      if (hasSynchronous()) {
+        result = result && (getSynchronous()
+            == other.getSynchronous());
+      }
+      result = result && getSwitchTypesList()
+          .equals(other.getSwitchTypesList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -28915,58 +29178,70 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasEnabled()) {
+        hash = (37 * hash) + ENABLED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getEnabled());
+      }
+      if (hasSynchronous()) {
+        hash = (37 * hash) + SYNCHRONOUS_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSynchronous());
+      }
+      if (getSwitchTypesCount() > 0) {
+        hash = (37 * hash) + SWITCH_TYPES_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnumList(getSwitchTypesList());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -28975,7 +29250,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -28987,24 +29262,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.NormalizeRequest}
+     * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledRequest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequestOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -29024,6 +29299,12 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        enabled_ = false;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        synchronous_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        switchTypes_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -29033,43 +29314,79 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.enabled_ = enabled_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.synchronous_ = synchronous_;
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          switchTypes_ = java.util.Collections.unmodifiableList(switchTypes_);
+          bitField0_ = (bitField0_ & ~0x00000004);
+        }
+        result.switchTypes_ = switchTypes_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.getDefaultInstance()) return this;
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.getDefaultInstance()) return this;
+        if (other.hasEnabled()) {
+          setEnabled(other.getEnabled());
+        }
+        if (other.hasSynchronous()) {
+          setSynchronous(other.getSynchronous());
+        }
+        if (!other.switchTypes_.isEmpty()) {
+          if (switchTypes_.isEmpty()) {
+            switchTypes_ = other.switchTypes_;
+            bitField0_ = (bitField0_ & ~0x00000004);
+          } else {
+            ensureSwitchTypesIsMutable();
+            switchTypes_.addAll(other.switchTypes_);
+          }
+          onChanged();
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
+        if (!hasEnabled()) {
+          
+          return false;
+        }
         return true;
       }
 
@@ -29077,11 +29394,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -29090,50 +29407,193 @@ public final class MasterProtos {
         }
         return this;
       }
+      private int bitField0_;
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.NormalizeRequest)
+      // required bool enabled = 1;
+      private boolean enabled_ ;
+      /**
+       * <code>required bool enabled = 1;</code>
+       */
+      public boolean hasEnabled() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required bool enabled = 1;</code>
+       */
+      public boolean getEnabled() {
+        return enabled_;
+      }
+      /**
+       * <code>required bool enabled = 1;</code>
+       */
+      public Builder setEnabled(boolean value) {
+        bitField0_ |= 0x00000001;
+        enabled_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool enabled = 1;</code>
+       */
+      public Builder clearEnabled() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        enabled_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional bool synchronous = 2;
+      private boolean synchronous_ ;
+      /**
+       * <code>optional bool synchronous = 2;</code>
+       */
+      public boolean hasSynchronous() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool synchronous = 2;</code>
+       */
+      public boolean getSynchronous() {
+        return synchronous_;
+      }
+      /**
+       * <code>optional bool synchronous = 2;</code>
+       */
+      public Builder setSynchronous(boolean value) {
+        bitField0_ |= 0x00000002;
+        synchronous_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool synchronous = 2;</code>
+       */
+      public Builder clearSynchronous() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        synchronous_ = false;
+        onChanged();
+        return this;
+      }
+
+      // repeated .hbase.pb.MasterSwitchType switch_types = 3;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> switchTypes_ =
+        java.util.Collections.emptyList();
+      private void ensureSwitchTypesIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          switchTypes_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType>(switchTypes_);
+          bitField0_ |= 0x00000004;
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> getSwitchTypesList() {
+        return java.util.Collections.unmodifiableList(switchTypes_);
+      }
+      /**
+       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+       */
+      public int getSwitchTypesCount() {
+        return switchTypes_.size();
+      }
+      /**
+       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchTypes(int index) {
+        return switchTypes_.get(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+       */
+      public Builder setSwitchTypes(
+          int index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        ensureSwitchTypesIsMutable();
+        switchTypes_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+       */
+      public Builder addSwitchTypes(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        ensureSwitchTypesIsMutable();
+        switchTypes_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+       */
+      public Builder addAllSwitchTypes(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> values) {
+        ensureSwitchTypesIsMutable();
+        super.addAll(values, switchTypes_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
+       */
+      public Builder clearSwitchTypes() {
+        switchTypes_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000004);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SetSplitOrMergeEnabledRequest)
     }
 
     static {
-      defaultInstance = new NormalizeRequest(true);
+      defaultInstance = new SetSplitOrMergeEnabledRequest(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.NormalizeRequest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.SetSplitOrMergeEnabledRequest)
   }
 
-  public interface NormalizeResponseOrBuilder
+  public interface SetSplitOrMergeEnabledResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required bool normalizer_ran = 1;
+    // repeated bool prev_value = 1;
     /**
-     * <code>required bool normalizer_ran = 1;</code>
+     * <code>repeated bool prev_value = 1;</code>
      */
-    boolean hasNormalizerRan();
+    java.util.List<java.lang.Boolean> getPrevValueList();
     /**
-     * <code>required bool normalizer_ran = 1;</code>
+     * <code>repeated bool prev_value = 1;</code>
      */
-    boolean getNormalizerRan();
+    int getPrevValueCount();
+    /**
+     * <code>repeated bool prev_value = 1;</code>
+     */
+    boolean getPrevValue(int index);
   }
   /**
-   * Protobuf type {@code hbase.pb.NormalizeResponse}
+   * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledResponse}
    */
-  public static final class NormalizeResponse extends
+  public static final class SetSplitOrMergeEnabledResponse extends
       com.google.protobuf.GeneratedMessage
-      implements NormalizeResponseOrBuilder {
-    // Use NormalizeResponse.newBuilder() to construct.
-    private NormalizeResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements SetSplitOrMergeEnabledResponseOrBuilder {
+    // Use SetSplitOrMergeEnabledResponse.newBuilder() to construct.
+    private SetSplitOrMergeEnabledResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private NormalizeResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private SetSplitOrMergeEnabledResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final NormalizeResponse defaultInstance;
-    public static NormalizeResponse getDefaultInstance() {
+    private static final SetSplitOrMergeEnabledResponse defaultInstance;
+    public static SetSplitOrMergeEnabledResponse getDefaultInstance() {
       return defaultInstance;
     }
 
-    public NormalizeResponse getDefaultInstanceForType() {
+    public SetSplitOrMergeEnabledResponse getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -29143,7 +29603,7 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private NormalizeResponse(
+    private SetSplitOrMergeEnabledResponse(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -29167,8 +29627,24 @@ public final class MasterProtos {
               break;
             }
             case 8: {
-              bitField0_ |= 0x00000001;
-              normalizerRan_ = input.readBool();
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                prevValue_ = new java.util.ArrayList<java.lang.Boolean>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              prevValue_.add(input.readBool());
+              break;
+            }
+            case 10: {
+              int length = input.readRawVarint32();
+              int limit = input.pushLimit(length);
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001) && input.getBytesUntilLimit() > 0) {
+                prevValue_ = new java.util.ArrayList<java.lang.Boolean>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              while (input.getBytesUntilLimit() > 0) {
+                prevValue_.add(input.readBool());
+              }
+              input.popLimit(limit);
               break;
             }
           }
@@ -29179,66 +29655,71 @@ public final class MasterProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          prevValue_ = java.util.Collections.unmodifiableList(prevValue_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<NormalizeResponse> PARSER =
-        new com.google.protobuf.AbstractParser<NormalizeResponse>() {
-      public NormalizeResponse parsePartialFrom(
+    public static com.google.protobuf.Parser<SetSplitOrMergeEnabledResponse> PARSER =
+        new com.google.protobuf.AbstractParser<SetSplitOrMergeEnabledResponse>() {
+      public SetSplitOrMergeEnabledResponse parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new NormalizeResponse(input, extensionRegistry);
+        return new SetSplitOrMergeEnabledResponse(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<NormalizeResponse> getParserForType() {
+    public com.google.protobuf.Parser<SetSplitOrMergeEnabledResponse> getParserForType() {
       return PARSER;
     }
 
-    private int bitField0_;
-    // required bool normalizer_ran = 1;
-    public static final int NORMALIZER_RAN_FIELD_NUMBER = 1;
-    private boolean normalizerRan_;
+    // repeated bool prev_value = 1;
+    public static final int PREV_VALUE_FIELD_NUMBER = 1;
+    private java.util.List<java.lang.Boolean> prevValue_;
     /**
-     * <code>required bool normalizer_ran = 1;</code>
+     * <code>repeated bool prev_value = 1;</code>
      */
-    public boolean hasNormalizerRan() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
+    public java.util.List<java.lang.Boolean>
+        getPrevValueList() {
+      return prevValue_;
     }
     /**
-     * <code>required bool normalizer_ran = 1;</code>
+     * <code>repeated bool prev_value = 1;</code>
      */
-    public boolean getNormalizerRan() {
-      return normalizerRan_;
+    public int getPrevValueCount() {
+      return prevValue_.size();
+    }
+    /**
+     * <code>repeated bool prev_value = 1;</code>
+     */
+    public boolean getPrevValue(int index) {
+      return prevValue_.get(index);
     }
 
     private void initFields() {
-      normalizerRan_ = false;
+      prevValue_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasNormalizerRan()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -29246,8 +29727,8 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBool(1, normalizerRan_);
+      for (int i = 0; i < prevValue_.size(); i++) {
+        output.writeBool(1, prevValue_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -29258,9 +29739,11 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(1, normalizerRan_);
+      {
+        int dataSize = 0;
+        dataSize = 1 * getPrevValueList().size();
+        size += dataSize;
+        size += 1 * getPrevValueList().size();
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -29279,17 +29762,14 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) obj;
 
       boolean result = true;
-      result = result && (hasNormalizerRan() == other.hasNormalizerRan());
-      if (hasNormalizerRan()) {
-        result = result && (getNormalizerRan()
-            == other.getNormalizerRan());
-      }
+      result = result && getPrevValueList()
+          .equals(other.getPrevValueList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -29303,62 +29783,62 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasNormalizerRan()) {
-        hash = (37 * hash) + NORMALIZER_RAN_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getNormalizerRan());
+      if (getPrevValueCount() > 0) {
+        hash = (37 * hash) + PREV_VALUE_FIELD_NUMBER;
+        hash = (53 * hash) + getPrevValueList().hashCode();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -29367,7 +29847,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -29379,24 +29859,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.NormalizeResponse}
+     * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledResponse}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponseOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponseOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -29416,7 +29896,7 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
-        normalizerRan_ = false;
+        prevValue_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
@@ -29427,57 +29907,59 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse(this);
         int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          prevValue_ = java.util.Collections.unmodifiableList(prevValue_);
+          bitField0_ = (bitField0_ & ~0x00000001);
         }
-        result.normalizerRan_ = normalizerRan_;
-        result.bitField0_ = to_bitField0_;
+        result.prevValue_ = prevValue_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.getDefaultInstance()) return this;
-        if (other.hasNormalizerRan()) {
-          setNormalizerRan(other.getNormalizerRan());
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.getDefaultInstance()) return this;
+        if (!other.prevValue_.isEmpty()) {
+          if (prevValue_.isEmpty()) {
+            prevValue_ = other.prevValue_;
+            bitField0_ = (bitField0_ & ~0x00000001);
+          } else {
+            ensurePrevValueIsMutable();
+            prevValue_.addAll(other.prevValue_);
+          }
+          onChanged();
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
-        if (!hasNormalizerRan()) {
-          
-          return false;
-        }
         return true;
       }
 
@@ -29485,11 +29967,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -29500,82 +29982,115 @@ public final class MasterProtos {
       }
       private int bitField0_;
 
-      // required bool normalizer_ran = 1;
-      private boolean normalizerRan_ ;
+      // repeated bool prev_value = 1;
+      private java.util.List<java.lang.Boolean> prevValue_ = java.util.Collections.emptyList();
+      private void ensurePrevValueIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          prevValue_ = new java.util.ArrayList<java.lang.Boolean>(prevValue_);
+          bitField0_ |= 0x00000001;
+         }
+      }
       /**
-       * <code>required bool normalizer_ran = 1;</code>
+       * <code>repeated bool prev_value = 1;</code>
        */
-      public boolean hasNormalizerRan() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
+      public java.util.List<java.lang.Boolean>
+          getPrevValueList() {
+        return java.util.Collections.unmodifiableList(prevValue_);
       }
       /**
-       * <code>required bool normalizer_ran = 1;</code>
+       * <code>repeated bool prev_value = 1;</code>
        */
-      public boolean getNormalizerRan() {
-        return normalizerRan_;
+      public int getPrevValueCount() {
+        return prevValue_.size();
       }
       /**
-       * <code>required bool normalizer_ran = 1;</code>
+       * <code>repeated bool prev_value = 1;</code>
        */
-      public Builder setNormalizerRan(boolean value) {
-        bitField0_ |= 0x00000001;
-        normalizerRan_ = value;
+      public boolean getPrevValue(int index) {
+        return prevValue_.get(index);
+      }
+      /**
+       * <code>repeated bool prev_value = 1;</code>
+       */
+      public Builder setPrevValue(
+          int index, boolean value) {
+        ensurePrevValueIsMutable();
+        prevValue_.set(index, value);
         onChanged();
         return this;
       }
       /**
-       * <code>required bool normalizer_ran = 1;</code>
+       * <code>repeated bool prev_value = 1;</code>
        */
-      public Builder clearNormalizerRan() {
+      public Builder addPrevValue(boolean value) {
+        ensurePrevValueIsMutable();
+        prevValue_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bool prev_value = 1;</code>
+       */
+      public Builder addAllPrevValue(
+          java.lang.Iterable<? extends java.lang.Boolean> values) {
+        ensurePrevValueIsMutable();
+        super.addAll(values, prevValue_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bool prev_value = 1;</code>
+       */
+      public Builder clearPrevValue() {
+        prevValue_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
-        normalizerRan_ = false;
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.NormalizeResponse)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SetSplitOrMergeEnabledResponse)
     }
 
     static {
-      defaultInstance = new NormalizeResponse(true);
+      defaultInstance = new SetSplitOrMergeEnabledResponse(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.NormalizeResponse)
+    // @@protoc_insertion_point(class_scope:hbase.pb.SetSplitOrMergeEnabledResponse)
   }
 
-  public interface SetNormalizerRunningRequestOrBuilder
+  public interface IsSplitOrMergeEnabledRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required bool on = 1;
+    // required .hbase.pb.MasterSwitchType switch_type = 1;
     /**
-     * <code>required bool on = 1;</code>
+     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
      */
-    boolean hasOn();
+    boolean hasSwitchType();
     /**
-     * <code>required bool on = 1;</code>
+     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
      */
-    boolean getOn();
+    org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchType();
   }
   /**
-   * Protobuf type {@code hbase.pb.SetNormalizerRunningRequest}
+   * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledRequest}
    */
-  public static final class SetNormalizerRunningRequest extends
+  public static final class IsSplitOrMergeEnabledRequest extends
       com.google.protobuf.GeneratedMessage
-      implements SetNormalizerRunningRequestOrBuilder {
-    // Use SetNormalizerRunningRequest.newBuilder() to construct.
-    private SetNormalizerRunningRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements IsSplitOrMergeEnabledRequestOrBuilder {
+    // Use IsSplitOrMergeEnabledRequest.newBuilder() to construct.
+    private IsSplitOrMergeEnabledRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private SetNormalizerRunningRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private IsSplitOrMergeEnabledRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final SetNormalizerRunningRequest defaultInstance;
-    public static SetNormalizerRunningRequest getDefaultInstance() {
+    private static final IsSplitOrMergeEnabledRequest defaultInstance;
+    public static IsSplitOrMergeEnabledRequest getDefaultInstance() {
       return defaultInstance;
     }
 
-    public SetNormalizerRunningRequest getDefaultInstanceForType() {
+    public IsSplitOrMergeEnabledRequest getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -29585,7 +30100,7 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private SetNormalizerRunningRequest(
+    private IsSplitOrMergeEnabledRequest(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -29609,8 +30124,14 @@ public final class MasterProtos {
               break;
             }
             case 8: {
-              bitField0_ |= 0x00000001;
-              on_ = input.readBool();
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                switchType_ = value;
+              }
               break;
             }
           }
@@ -29627,57 +30148,57 @@ public final class MasterProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<SetNormalizerRunningRequest> PARSER =
-        new com.google.protobuf.AbstractParser<SetNormalizerRunningRequest>() {
-      public SetNormalizerRunningRequest parsePartialFrom(
+    public static com.google.protobuf.Parser<IsSplitOrMergeEnabledRequest> PARSER =
+        new com.google.protobuf.AbstractParser<IsSplitOrMergeEnabledRequest>() {
+      public IsSplitOrMergeEnabledRequest parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new SetNormalizerRunningRequest(input, extensionRegistry);
+        return new IsSplitOrMergeEnabledRequest(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<SetNormalizerRunningRequest> getParserForType() {
+    public com.google.protobuf.Parser<IsSplitOrMergeEnabledRequest> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // required bool on = 1;
-    public static final int ON_FIELD_NUMBER = 1;
-    private boolean on_;
+    // required .hbase.pb.MasterSwitchType switch_type = 1;
+    public static final int SWITCH_TYPE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType switchType_;
     /**
-     * <code>required bool on = 1;</code>
+     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
      */
-    public boolean hasOn() {
+    public boolean hasSwitchType() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required bool on = 1;</code>
+     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
      */
-    public boolean getOn() {
-      return on_;
+    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchType() {
+      return switchType_;
     }
 
     private void initFields() {
-      on_ = false;
+      switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasOn()) {
+      if (!hasSwitchType()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -29689,7 +30210,7 @@ public final class MasterProtos {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBool(1, on_);
+        output.writeEnum(1, switchType_.getNumber());
       }
       getUnknownFields().writeTo(output);
     }
@@ -29702,7 +30223,7 @@ public final class MasterProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(1, on_);
+          .computeEnumSize(1, switchType_.getNumber());
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -29721,16 +30242,16 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) obj;
 
       boolean result = true;
-      result = result && (hasOn() == other.hasOn());
-      if (hasOn()) {
-        result = result && (getOn()
-            == other.getOn());
+      result = result && (hasSwitchType() == other.hasSwitchType());
+      if (hasSwitchType()) {
+        result = result &&
+            (getSwitchType() == other.getSwitchType());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -29745,62 +30266,62 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasOn()) {
-        hash = (37 * hash) + ON_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getOn());
+      if (hasSwitchType()) {
+        hash = (37 * hash) + SWITCH_TYPE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getSwitchType());
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -29809,7 +30330,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -29821,24 +30342,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.SetNormalizerRunningRequest}
+     * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledRequest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequestOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -29858,7 +30379,7 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
-        on_ = false;
+        switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
@@ -29869,54 +30390,54 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.on_ = on_;
+        result.switchType_ = switchType_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.getDefaultInstance()) return this;
-        if (other.hasOn()) {
-          setOn(other.getOn());
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.getDefaultInstance()) return this;
+        if (other.hasSwitchType()) {
+          setSwitchType(other.getSwitchType());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
-        if (!hasOn()) {
+        if (!hasSwitchType()) {
           
           return false;
         }
@@ -29927,11 +30448,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -29942,82 +30463,85 @@ public final class MasterProtos {
       }
       private int bitField0_;
 
-      // required bool on = 1;
-      private boolean on_ ;
+      // required .hbase.pb.MasterSwitchType switch_type = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
       /**
-       * <code>required bool on = 1;</code>
+       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
        */
-      public boolean hasOn() {
+      public boolean hasSwitchType() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required bool on = 1;</code>
+       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
        */
-      public boolean getOn() {
-        return on_;
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchType() {
+        return switchType_;
       }
       /**
-       * <code>required bool on = 1;</code>
+       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
        */
-      public Builder setOn(boolean value) {
+      public Builder setSwitchType(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
         bitField0_ |= 0x00000001;
-        on_ = value;
+        switchType_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bool on = 1;</code>
+       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
        */
-      public Builder clearOn() {
+      public Builder clearSwitchType() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        on_ = false;
+        switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SetNormalizerRunningRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.IsSplitOrMergeEnabledRequest)
     }
 
     static {
-      defaultInstance = new SetNormalizerRunningRequest(true);
+      defaultInstance = new IsSplitOrMergeEnabledRequest(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.SetNormalizerRunningRequest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.IsSplitOrMergeEnabledRequest)
   }
 
-  public interface SetNormalizerRunningResponseOrBuilder
+  public interface IsSplitOrMergeEnabledResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional bool prev_normalizer_value = 1;
+    // required bool enabled = 1;
     /**
-     * <code>optional bool prev_normalizer_value = 1;</code>
+     * <code>required bool enabled = 1;</code>
      */
-    boolean hasPrevNormalizerValue();
+    boolean hasEnabled();
     /**
-     * <code>optional bool prev_normalizer_value = 1;</code>
+     * <code>required bool enabled = 1;</code>
      */
-    boolean getPrevNormalizerValue();
+    boolean getEnabled();
   }
   /**
-   * Protobuf type {@code hbase.pb.SetNormalizerRunningResponse}
+   * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledResponse}
    */
-  public static final class SetNormalizerRunningResponse extends
+  public static final class IsSplitOrMergeEnabledResponse extends
       com.google.protobuf.GeneratedMessage
-      implements SetNormalizerRunningResponseOrBuilder {
-    // Use SetNormalizerRunningResponse.newBuilder() to construct.
-    private SetNormalizerRunningResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements IsSplitOrMergeEnabledResponseOrBuilder {
+    // Use IsSplitOrMergeEnabledResponse.newBuilder() to construct.
+    private IsSplitOrMergeEnabledResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private SetNormalizerRunningResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private IsSplitOrMergeEnabledResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final SetNormalizerRunningResponse defaultInstance;
-    public static SetNormalizerRunningResponse getDefaultInstance() {
+    private static final IsSplitOrMergeEnabledResponse defaultInstance;
+    public static IsSplitOrMergeEnabledResponse getDefaultInstance() {
       return defaultInstance;
     }
 
-    public SetNormalizerRunningResponse getDefaultInstanceForType() {
+    public IsSplitOrMergeEnabledRes

<TRUNCATED>

[19/37] hbase git commit: HBASE-15136 Explore different queuing behaviors while busy

Posted by sy...@apache.org.
HBASE-15136 Explore different queuing behaviors while busy


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

Branch: refs/heads/hbase-12439
Commit: 43f99def670551cfe314c44181c0cb9570cdaaa3
Parents: 6e9d355
Author: Mikhail Antonov <an...@apache.org>
Authored: Wed Feb 24 20:40:44 2016 -0800
Committer: Mikhail Antonov <an...@apache.org>
Committed: Wed Feb 24 20:41:30 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/ReflectionUtils.java      |   1 +
 .../hbase/ipc/MetricsHBaseServerSource.java     |   6 +
 .../hbase/ipc/MetricsHBaseServerWrapper.java    |   2 +
 .../hbase/ipc/MetricsHBaseServerSourceImpl.java |   6 +-
 .../hbase/ipc/AdaptiveLifoCoDelCallQueue.java   | 329 +++++++++++++++++++
 .../hadoop/hbase/ipc/FifoRpcScheduler.java      |  10 +
 .../ipc/MetricsHBaseServerWrapperImpl.java      |  16 +
 .../hadoop/hbase/ipc/RWQueueRpcExecutor.java    |  10 +
 .../apache/hadoop/hbase/ipc/RpcScheduler.java   |  13 +
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    |  70 +++-
 .../ipc/MetricsHBaseServerWrapperStub.java      |  10 +
 .../hbase/ipc/TestSimpleRpcScheduler.java       |  63 ++++
 12 files changed, 534 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
index 650c544..15b3930 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java
@@ -85,6 +85,7 @@ public class ReflectionUtils {
         match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType) :
                   ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
                    (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
+                   (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
                    (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
                    (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
                    (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
index 061a672..bb89789 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
@@ -64,6 +64,12 @@ public interface MetricsHBaseServerSource extends BaseSource {
   String NUM_OPEN_CONNECTIONS_DESC = "Number of open connections.";
   String NUM_ACTIVE_HANDLER_NAME = "numActiveHandler";
   String NUM_ACTIVE_HANDLER_DESC = "Number of active rpc handlers.";
+  String NUM_GENERAL_CALLS_DROPPED_NAME = "numGeneralCallsDropped";
+  String NUM_GENERAL_CALLS_DROPPED_DESC = "Total number of calls in general queue which " +
+    "were dropped by CoDel RPC executor";
+  String NUM_LIFO_MODE_SWITCHES_NAME = "numLifoModeSwitches";
+  String NUM_LIFO_MODE_SWITCHES_DESC = "Total number of calls in general queue which " +
+    "were served from the tail of the queue";
 
   String EXCEPTIONS_NAME="exceptions";
   String EXCEPTIONS_DESC="Exceptions caused by requests";

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java
index 1885264..8f30205 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java
@@ -26,4 +26,6 @@ public interface MetricsHBaseServerWrapper {
   int getPriorityQueueLength();
   int getNumOpenConnections();
   int getActiveRpcHandlerCount();
+  long getNumGeneralCallsDropped();
+  long getNumLifoModeSwitches();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
index 48f57e9..c466564 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
@@ -219,7 +219,11 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
           .addGauge(Interns.info(NUM_OPEN_CONNECTIONS_NAME,
               NUM_OPEN_CONNECTIONS_DESC), wrapper.getNumOpenConnections())
           .addGauge(Interns.info(NUM_ACTIVE_HANDLER_NAME,
-              NUM_ACTIVE_HANDLER_DESC), wrapper.getActiveRpcHandlerCount());
+              NUM_ACTIVE_HANDLER_DESC), wrapper.getActiveRpcHandlerCount())
+          .addCounter(Interns.info(NUM_GENERAL_CALLS_DROPPED_NAME,
+              NUM_GENERAL_CALLS_DROPPED_DESC), wrapper.getNumGeneralCallsDropped())
+          .addCounter(Interns.info(NUM_LIFO_MODE_SWITCHES_NAME,
+              NUM_LIFO_MODE_SWITCHES_DESC), wrapper.getNumLifoModeSwitches());
     }
 
     metricsRegistry.snapshot(mrb, all);

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java
new file mode 100644
index 0000000..37e86be
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java
@@ -0,0 +1,329 @@
+/**
+ * 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.ipc;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Adaptive LIFO blocking queue utilizing CoDel algorithm to prevent queue overloading.
+ *
+ * Implementing {@link BlockingQueue} interface to be compatible with {@link RpcExecutor}.
+ *
+ * Currently uses milliseconds internally, need to look into whether we should use
+ * nanoseconds for timeInterval and minDelay.
+ *
+ * @see <a href="http://queue.acm.org/detail.cfm?id=2839461">Fail at Scale paper</a>
+ *
+ * @see <a href="https://github.com/facebook/wangle/blob/master/wangle/concurrent/Codel.cpp">
+ *   CoDel version for generic job queues in Wangle library</a>
+ */
+@InterfaceAudience.Private
+public class AdaptiveLifoCoDelCallQueue implements BlockingQueue<CallRunner> {
+
+  // backing queue
+  private LinkedBlockingDeque<CallRunner> queue;
+
+  // so we can calculate actual threshold to switch to LIFO under load
+  private int maxCapacity;
+
+  // metrics (shared across all queues)
+  private AtomicLong numGeneralCallsDropped;
+  private AtomicLong numLifoModeSwitches;
+
+  /**
+   * Lock held by take ops, all other locks are inside queue impl.
+   *
+   * NOTE: We want to have this lock so that in case when there're lot of already expired
+   * calls in the call queue a handler thread calling take() can just grab lock once and
+   * then fast-forward through the expired calls to the first non-expired without having
+   * to contend for locks on every element in underlying queue.
+   */
+  private final ReentrantLock lock = new ReentrantLock();
+
+  // Both are in milliseconds
+  private volatile int codelTargetDelay;
+  private volatile int codelInterval;
+
+  // if queue if full more than that percent, we switch to LIFO mode.
+  // Values are in the range of 0.7, 0.8 etc (0-1.0).
+  private volatile double lifoThreshold;
+
+  // minimal delay observed during the interval
+  private volatile long minDelay;
+
+  // the moment when current interval ends
+  private volatile long intervalTime = System.currentTimeMillis();
+
+  // switch to ensure only one threads does interval cutoffs
+  private AtomicBoolean resetDelay = new AtomicBoolean(true);
+
+  // if we're in this mode, "long" calls are getting dropped
+  private volatile boolean isOverloaded;
+
+  public AdaptiveLifoCoDelCallQueue(int capacity, int targetDelay, int interval,
+      double lifoThreshold, AtomicLong numGeneralCallsDropped, AtomicLong numLifoModeSwitches) {
+    this.maxCapacity = capacity;
+    this.queue = new LinkedBlockingDeque<>(capacity);
+    this.codelTargetDelay = targetDelay;
+    this.codelInterval = interval;
+    this.lifoThreshold = lifoThreshold;
+    this.numGeneralCallsDropped = numGeneralCallsDropped;
+    this.numLifoModeSwitches = numLifoModeSwitches;
+  }
+
+  /**
+   * Update tunables.
+   *
+   * @param newCodelTargetDelay new CoDel target delay
+   * @param newCodelInterval new CoDel interval
+   * @param newLifoThreshold new Adaptive Lifo threshold
+   */
+  public void updateTunables(int newCodelTargetDelay, int newCodelInterval,
+                             double newLifoThreshold) {
+    this.codelTargetDelay = newCodelTargetDelay;
+    this.codelInterval = newCodelInterval;
+    this.lifoThreshold = newLifoThreshold;
+  }
+
+  /**
+   * Behaves as {@link LinkedBlockingQueue#take()}, except it will silently
+   * skip all calls which it thinks should be dropped.
+   *
+   * @return the head of this queue
+   * @throws InterruptedException if interrupted while waiting
+   */
+  @Override
+  public CallRunner take() throws InterruptedException {
+    final ReentrantLock lock = this.lock;
+    lock.lock();
+    try {
+      CallRunner cr;
+      while(true) {
+        if (((double) queue.size() / this.maxCapacity) > lifoThreshold) {
+          numLifoModeSwitches.incrementAndGet();
+          cr = queue.takeLast();
+        } else {
+          cr = queue.takeFirst();
+        }
+        if (needToDrop(cr)) {
+          numGeneralCallsDropped.incrementAndGet();
+        } else {
+          return cr;
+        }
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * @param callRunner to validate
+   * @return true if this call needs to be skipped based on call timestamp
+   *   and internal queue state (deemed overloaded).
+   */
+  private boolean needToDrop(CallRunner callRunner) {
+    long now = System.currentTimeMillis();
+    long callDelay = now - callRunner.getCall().timestamp;
+
+    long localMinDelay = this.minDelay;
+    if (now > intervalTime && !resetDelay.getAndSet(true)) {
+      intervalTime = now + codelInterval;
+
+      if (localMinDelay > codelTargetDelay) {
+        isOverloaded = true;
+      } else {
+        isOverloaded = false;
+      }
+    }
+
+    if (resetDelay.getAndSet(false)) {
+      minDelay = callDelay;
+      return false;
+    } else if (callDelay < localMinDelay) {
+      minDelay = callDelay;
+    }
+
+    if (isOverloaded && callDelay > 2 * codelTargetDelay) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  // Generic BlockingQueue methods we support
+  @Override
+  public boolean offer(CallRunner callRunner) {
+    return queue.offer(callRunner);
+  }
+
+  @Override
+  public int size() {
+    return queue.size();
+  }
+
+  @Override
+  public String toString() {
+    return queue.toString();
+  }
+
+  // This class does NOT provide generic purpose BlockingQueue implementation,
+  // so to prevent misuse all other methods throw UnsupportedOperationException.
+
+  @Override
+  public CallRunner poll(long timeout, TimeUnit unit) throws InterruptedException {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public CallRunner poll() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public CallRunner peek() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean remove(Object o) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public Object[] toArray() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public <T> T[] toArray(T[] a) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public void clear() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public int drainTo(Collection<? super CallRunner> c) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public int drainTo(Collection<? super CallRunner> c, int maxElements) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public Iterator<CallRunner> iterator() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean add(CallRunner callRunner) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public CallRunner remove() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public CallRunner element() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean addAll(Collection<? extends CallRunner> c) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean isEmpty() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> c) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean removeAll(Collection<?> c) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean retainAll(Collection<?> c) {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public int remainingCapacity() {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public void put(CallRunner callRunner) throws InterruptedException {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+
+  @Override
+  public boolean offer(CallRunner callRunner, long timeout, TimeUnit unit)
+      throws InterruptedException {
+    throw new UnsupportedOperationException("This class doesn't support anything,"
+      + " but take() and offer() methods");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
index 5e104eb..b069a5a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
@@ -104,4 +104,14 @@ public class FifoRpcScheduler extends RpcScheduler {
   public int getActiveRpcHandlerCount() {
     return executor.getActiveCount();
   }
+
+  @Override
+  public long getNumGeneralCallsDropped() {
+    return 0;
+  }
+
+  @Override
+  public long getNumLifoModeSwitches() {
+    return 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
index 63c4b32..9979c75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
@@ -78,4 +78,20 @@ public class MetricsHBaseServerWrapperImpl implements MetricsHBaseServerWrapper
     }
     return server.getScheduler().getActiveRpcHandlerCount();
   }
+
+  @Override
+  public long getNumGeneralCallsDropped() {
+    if (!isServerStarted() || this.server.getScheduler() == null) {
+      return 0;
+    }
+    return server.getScheduler().getNumGeneralCallsDropped();
+  }
+
+  @Override
+  public long getNumLifoModeSwitches() {
+    if (!isServerStarted() || this.server.getScheduler() == null) {
+      return 0;
+    }
+    return server.getScheduler().getNumLifoModeSwitches();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
index a9648b0..e0203ab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
@@ -100,6 +100,16 @@ public class RWQueueRpcExecutor extends RpcExecutor {
       readQueueClass, ArrayUtils.addAll(new Object[] {maxQueueLength}, readQueueInitArgs));
   }
 
+  public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
+      final float readShare, final float scanShare,
+      final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
+      final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
+    this(name, calcNumWriters(handlerCount, readShare), calcNumReaders(handlerCount, readShare),
+      calcNumWriters(numQueues, readShare), calcNumReaders(numQueues, readShare), scanShare,
+      writeQueueClass, writeQueueInitArgs,
+      readQueueClass, readQueueInitArgs);
+  }
+
   public RWQueueRpcExecutor(final String name, final int writeHandlers, final int readHandlers,
       final int numWriteQueues, final int numReadQueues,
       final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
index fffe7f3..50886cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
@@ -71,4 +71,17 @@ public abstract class RpcScheduler {
 
   /** Retrieves the number of active handler. */
   public abstract int getActiveRpcHandlerCount();
+
+  /**
+   * If CoDel-based RPC executors are used, retrieves the number of Calls that were dropped
+   * from general queue because RPC executor is under high load; returns 0 otherwise.
+   */
+  public abstract long getNumGeneralCallsDropped();
+
+  /**
+   * If CoDel-based RPC executors are used, retrieves the number of Calls that were
+   * picked from the tail of the queue (indicating adaptive LIFO mode, when
+   * in the period of overloade we serve last requests first); returns 0 otherwise.
+   */
+  public abstract long getNumLifoModeSwitches();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index 0003254..12ee540 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.ipc;
 
 
 import java.util.Comparator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,6 +51,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
 
   /** If set to 'deadline', uses a priority queue and deprioritize long-running scans */
   public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
+  public static final String CALL_QUEUE_TYPE_CODEL_CONF_VALUE = "codel";
   public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
   public static final String CALL_QUEUE_TYPE_FIFO_CONF_VALUE = "fifo";
 
@@ -56,6 +59,21 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
   public static final String QUEUE_MAX_CALL_DELAY_CONF_KEY
       = "hbase.ipc.server.queue.max.call.delay";
 
+  // These 3 are only used by Codel executor
+  public static final String CALL_QUEUE_CODEL_TARGET_DELAY =
+    "hbase.ipc.server.callqueue.codel.target.delay";
+  public static final String CALL_QUEUE_CODEL_INTERVAL =
+    "hbase.ipc.server.callqueue.codel.interval";
+  public static final String CALL_QUEUE_CODEL_LIFO_THRESHOLD =
+    "hbase.ipc.server.callqueue.codel.lifo.threshold";
+
+  public static final int CALL_QUEUE_CODEL_DEFAULT_TARGET_DELAY = 5;
+  public static final int CALL_QUEUE_CODEL_DEFAULT_INTERVAL = 100;
+  public static final double CALL_QUEUE_CODEL_DEFAULT_LIFO_THRESHOLD = 0.8;
+
+  private AtomicLong numGeneralCallsDropped = new AtomicLong();
+  private AtomicLong numLifoModeSwitches = new AtomicLong();
+
   /**
    * Resize call queues;
    * @param conf new configuration
@@ -69,6 +87,26 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     if (replicationExecutor != null) {
       replicationExecutor.resizeQueues(conf);
     }
+
+    String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY,
+      CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
+
+    if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
+      // update CoDel Scheduler tunables
+      int codelTargetDelay = conf.getInt(CALL_QUEUE_CODEL_TARGET_DELAY,
+        CALL_QUEUE_CODEL_DEFAULT_TARGET_DELAY);
+      int codelInterval = conf.getInt(CALL_QUEUE_CODEL_INTERVAL,
+        CALL_QUEUE_CODEL_DEFAULT_INTERVAL);
+      double codelLifoThreshold = conf.getDouble(CALL_QUEUE_CODEL_LIFO_THRESHOLD,
+        CALL_QUEUE_CODEL_DEFAULT_LIFO_THRESHOLD);
+
+      for (BlockingQueue<CallRunner> queue : callExecutor.getQueues()) {
+        if (queue instanceof AdaptiveLifoCoDelCallQueue) {
+          ((AdaptiveLifoCoDelCallQueue) queue).updateTunables(codelTargetDelay,
+            codelInterval, codelLifoThreshold);
+        }
+      }
+    }
   }
 
   /**
@@ -134,10 +172,18 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     this.highPriorityLevel = highPriorityLevel;
     this.abortable = server;
 
-    String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY, CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
+    String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY,
+      CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
     float callqReadShare = conf.getFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0);
     float callqScanShare = conf.getFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0);
 
+    int codelTargetDelay = conf.getInt(CALL_QUEUE_CODEL_TARGET_DELAY,
+      CALL_QUEUE_CODEL_DEFAULT_TARGET_DELAY);
+    int codelInterval = conf.getInt(CALL_QUEUE_CODEL_INTERVAL,
+      CALL_QUEUE_CODEL_DEFAULT_INTERVAL);
+    double codelLifoThreshold = conf.getDouble(CALL_QUEUE_CODEL_LIFO_THRESHOLD,
+      CALL_QUEUE_CODEL_DEFAULT_LIFO_THRESHOLD);
+
     float callQueuesHandlersFactor = conf.getFloat(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0);
     int numCallQueues = Math.max(1, (int)Math.round(handlerCount * callQueuesHandlersFactor));
 
@@ -150,6 +196,13 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
         callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount, numCallQueues,
             callqReadShare, callqScanShare, maxQueueLength, conf, abortable,
             BoundedPriorityBlockingQueue.class, callPriority);
+      } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
+        Object[] callQueueInitArgs = {maxQueueLength, codelTargetDelay, codelInterval,
+          codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches};
+        callExecutor = new RWQueueRpcExecutor("B.default", handlerCount,
+          numCallQueues, callqReadShare, callqScanShare,
+          AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs,
+          AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs);
       } else {
         callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount, numCallQueues,
           callqReadShare, callqScanShare, maxQueueLength, conf, abortable);
@@ -160,6 +213,11 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
         CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
         callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount, numCallQueues,
           conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
+      } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
+        callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount, numCallQueues,
+          conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
+          codelTargetDelay, codelInterval, codelLifoThreshold,
+          numGeneralCallsDropped, numLifoModeSwitches);
       } else {
         callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount,
             numCallQueues, maxQueueLength, conf, abortable);
@@ -239,5 +297,15 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
            (priorityExecutor == null ? 0 : priorityExecutor.getActiveHandlerCount()) +
            (replicationExecutor == null ? 0 : replicationExecutor.getActiveHandlerCount());
   }
+
+  @Override
+  public long getNumGeneralCallsDropped() {
+    return numGeneralCallsDropped.get();
+  }
+
+  @Override
+  public long getNumLifoModeSwitches() {
+    return numLifoModeSwitches.get();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java
index 6241f8e..b001d74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java
@@ -48,4 +48,14 @@ public class MetricsHBaseServerWrapperStub implements MetricsHBaseServerWrapper{
   public int getActiveRpcHandlerCount() {
     return 106;
   }
+
+  @Override
+  public long getNumGeneralCallsDropped() {
+    return 3;
+  }
+
+  @Override
+  public long getNumLifoModeSwitches() {
+    return 5;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/43f99def/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 66032e9..916037b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -362,4 +362,67 @@ public class TestSimpleRpcScheduler {
       scheduler.stop();
     }
   }
+
+  @Test
+  public void testCoDelScheduling() throws Exception {
+    Configuration schedConf = HBaseConfiguration.create();
+
+    schedConf.set(SimpleRpcScheduler.CALL_QUEUE_TYPE_CONF_KEY,
+      SimpleRpcScheduler.CALL_QUEUE_TYPE_CODEL_CONF_VALUE);
+
+    PriorityFunction priority = mock(PriorityFunction.class);
+    when(priority.getPriority(any(RPCProtos.RequestHeader.class), any(Message.class),
+      any(User.class))).thenReturn(HConstants.NORMAL_QOS);
+    SimpleRpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 1, 1, 1, priority,
+      HConstants.QOS_THRESHOLD);
+    try {
+      scheduler.start();
+
+      // calls faster than min delay
+      for (int i = 0; i < 100; i++) {
+        CallRunner cr = getMockedCallRunner();
+        Thread.sleep(5);
+        scheduler.dispatch(cr);
+      }
+      Thread.sleep(100); // make sure fast calls are handled
+      assertEquals("None of these calls should have been discarded", 0,
+        scheduler.getNumGeneralCallsDropped());
+
+      // calls slower than min delay, but not individually slow enough to be dropped
+      for (int i = 0; i < 20; i++) {
+        CallRunner cr = getMockedCallRunner();
+        Thread.sleep(6);
+        scheduler.dispatch(cr);
+      }
+
+      Thread.sleep(100); // make sure somewhat slow calls are handled
+      assertEquals("None of these calls should have been discarded", 0,
+        scheduler.getNumGeneralCallsDropped());
+
+      // now slow calls and the ones to be dropped
+      for (int i = 0; i < 20; i++) {
+        CallRunner cr = getMockedCallRunner();
+        Thread.sleep(12);
+        scheduler.dispatch(cr);
+      }
+
+      Thread.sleep(100); // make sure somewhat slow calls are handled
+      assertTrue("There should have been at least 12 calls dropped",
+        scheduler.getNumGeneralCallsDropped() > 12);
+    } finally {
+      scheduler.stop();
+    }
+  }
+
+  private CallRunner getMockedCallRunner() throws IOException {
+    CallRunner putCallTask = mock(CallRunner.class);
+    RpcServer.Call putCall = mock(RpcServer.Call.class);
+    putCall.param = RequestConverter.buildMutateRequest(
+      Bytes.toBytes("abc"), new Put(Bytes.toBytes("row")));
+    RPCProtos.RequestHeader putHead = RPCProtos.RequestHeader.newBuilder().setMethodName("mutate").build();
+    when(putCallTask.getCall()).thenReturn(putCall);
+    when(putCall.getHeader()).thenReturn(putHead);
+    putCall.timestamp = System.currentTimeMillis();
+    return putCallTask;
+  }
 }


[15/37] hbase git commit: HBASE-15222 Use less contended classes for metrics

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java
index 32d4fae..aaf4359 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.metrics2.lib;
 
-import java.util.concurrent.atomic.AtomicLongArray;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.metrics2.MetricsInfo;
 
@@ -28,31 +26,30 @@ import org.apache.hadoop.metrics2.MetricsInfo;
  */
 @InterfaceAudience.Private
 public class MutableTimeHistogram extends MutableRangeHistogram {
-  private final String rangeType = "TimeRangeCount";
-  private final long[] ranges =
+  private final static String RANGE_TYPE = "TimeRangeCount";
+  private final static long[] RANGES =
       { 1, 3, 10, 30, 100, 300, 1000, 3000, 10000, 30000, 60000, 120000, 300000, 600000 };
-  private final AtomicLongArray rangeVals = new AtomicLongArray(ranges.length+1);
 
   public MutableTimeHistogram(MetricsInfo info) {
     this(info.name(), info.description());
   }
 
   public MutableTimeHistogram(String name, String description) {
-    super(name, description);
+    this(name, description, RANGES[RANGES.length - 2]);
+  }
+
+  public MutableTimeHistogram(String name, String description, long expectedMax) {
+    super(name, description, expectedMax);
   }
 
   @Override
   public String getRangeType() {
-    return rangeType;
+    return RANGE_TYPE;
   }
 
   @Override
-  public long[] getRange() {
-    return ranges;
+  public long[] getRanges() {
+    return RANGES;
   }
 
-  @Override
-  public AtomicLongArray getRangeVals() {
-    return rangeVals;
-  } 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
index 7381fb9..2e374f7 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.metrics;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.testclassification.MetricsTests;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -72,9 +73,9 @@ public class TestBaseSourceImpl {
   @Test
   public void testIncCounters() throws Exception {
     bmsi.incCounters("testinccounter", 100);
-    assertEquals(100, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
+    assertEquals(100, ((MutableFastCounter) bmsi.metricsRegistry.get("testinccounter")).value());
     bmsi.incCounters("testinccounter", 100);
-    assertEquals(200, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
+    assertEquals(200, ((MutableFastCounter) bmsi.metricsRegistry.get("testinccounter")).value());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
index 6986f12..3dcd5e2 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
@@ -178,7 +178,6 @@ org.apache.hadoop.util.StringUtils;
   AgeSnapshot ageAtEvictionSnapshot = bc.getStats().getAgeAtEvictionSnapshot();
   // Only show if non-zero mean and stddev as is the case in combinedblockcache
   double mean = ageAtEvictionSnapshot.getMean();
-  double stddev = ageAtEvictionSnapshot.getStdDev();
 </%java>
     <tr>
         <td>Evicted</td>
@@ -197,13 +196,6 @@ org.apache.hadoop.util.StringUtils;
         <td>Mean age of Blocks at eviction time (seconds)</td>
     </tr>
 </%if>
-<%if stddev > 0 %>
-    <tr>
-        <td>StdDev</td>
-        <td><% String.format("%,d", (long)(ageAtEvictionSnapshot.getStdDev()/1000000)) %></td>
-        <td>Standard Deviation for age of Blocks at eviction time</td>
-    </tr>
-</%if>
 </%def>
 
 <%def hits_tmpl>

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
index c23cf75..fa55f6a 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
@@ -37,7 +37,6 @@ org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator;
 org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket;
 org.apache.hadoop.util.StringUtils;
-com.codahale.metrics.Snapshot;
 </%import>
 <%java>
   BlockCache bc = cacheConfig == null ? null : cacheConfig.getBlockCache();

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
index a55a863..de2aeca 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
@@ -34,7 +34,6 @@ org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
 org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
 org.apache.hadoop.hbase.util.DirectMemoryUtils;
 org.apache.hadoop.util.StringUtils;
-com.codahale.metrics.Snapshot;
 java.lang.management.ManagementFactory;
 </%import>
 <div class="tabbable">

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
index 79acec0..4c1ad23 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
@@ -17,56 +17,54 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
+import org.apache.hadoop.hbase.util.FastLongHistogram;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.Snapshot;
-
 /**
  * Snapshot of block cache age in cache.
  * This object is preferred because we can control how it is serialized out when JSON'ing.
  */
 @JsonIgnoreProperties({"ageHistogram", "snapshot"})
 public class AgeSnapshot {
-  private final Snapshot snapshot;
 
-  AgeSnapshot(final Histogram ageHistogram) {
-    this.snapshot = ageHistogram.getSnapshot();
+  private final FastLongHistogram ageHistogram;
+  private final long[] quantiles;
+
+  AgeSnapshot(final FastLongHistogram ageHistogram) {
+    this.ageHistogram = ageHistogram;
+    this.quantiles = ageHistogram.getQuantiles(new double[]{0.75, 0.95, 0.98, 0.99, 0.999});
   }
 
   public double get75thPercentile() {
-    return snapshot.get75thPercentile();
+    return quantiles[0];
   }
 
   public double get95thPercentile() {
-    return snapshot.get95thPercentile();
+    return quantiles[1];
   }
 
   public double get98thPercentile() {
-    return snapshot.get98thPercentile();
+    return quantiles[2];
   }
 
-  public double get999thPercentile() {
-    return snapshot.get999thPercentile();
+  public double get99thPercentile() {
+    return quantiles[3];
   }
 
-  public double get99thPercentile() {
-    return snapshot.get99thPercentile();
+  public double get999thPercentile() {
+    return quantiles[4];
   }
 
+
   public double getMean() {
-    return this.snapshot.getMean();
+    return this.ageHistogram.getMean();
   }
 
   public double getMax() {
-    return snapshot.getMax();
+    return this.ageHistogram.getMax();
   }
 
   public double getMin() {
-    return snapshot.getMin();
-  }
-
-  public double getStdDev() {
-    return snapshot.getStdDev();
+    return this.ageHistogram.getMin();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
index d81871f..977284b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
@@ -25,17 +25,13 @@ import java.util.concurrent.ConcurrentSkipListSet;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.FastLongHistogram;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
 
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.Snapshot;
-
-import static com.codahale.metrics.MetricRegistry.name;
 
 /**
  * Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
@@ -44,11 +40,6 @@ import static com.codahale.metrics.MetricRegistry.name;
 @InterfaceAudience.Private
 public class BlockCacheUtil {
   /**
-   * Needed making histograms.
-   */
-  private static final MetricRegistry METRICS = new MetricRegistry();
-
-  /**
    * Needed generating JSON.
    */
   private static final ObjectMapper MAPPER = new ObjectMapper();
@@ -206,7 +197,7 @@ public class BlockCacheUtil {
      */
     private NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
       new ConcurrentSkipListMap<String, NavigableSet<CachedBlock>>();
-    Histogram age = METRICS.histogram(name(CachedBlocksByFile.class, "age"));
+    FastLongHistogram hist = new FastLongHistogram();
 
     /**
      * @param cb
@@ -228,7 +219,7 @@ public class BlockCacheUtil {
         this.dataSize += cb.getSize();
       }
       long age = this.now - cb.getCachedTime();
-      this.age.update(age);
+      this.hist.add(age, 1);
       return false;
     }
 
@@ -271,18 +262,22 @@ public class BlockCacheUtil {
     }
 
     public AgeSnapshot getAgeInCacheSnapshot() {
-      return new AgeSnapshot(this.age);
+      return new AgeSnapshot(this.hist);
     }
 
     @Override
     public String toString() {
-      Snapshot snapshot = age.getSnapshot();
+      AgeSnapshot snapshot = getAgeInCacheSnapshot();
       return "count=" + count + ", dataBlockCount=" + dataBlockCount + ", size=" + size +
           ", dataSize=" + getDataSize() +
-          ", mean age=" + snapshot.getMean() + ", stddev age=" + snapshot.getStdDev() +
-          ", min age=" + snapshot.getMin() + ", max age=" + snapshot.getMax() +
-          ", 95th percentile age=" + snapshot.get95thPercentile() +
-          ", 99th percentile age=" + snapshot.get99thPercentile();
+          ", mean age=" + snapshot.getMean() +
+          ", min age=" + snapshot.getMin() +
+          ", max age=" + snapshot.getMax() +
+          ", 75th percentile age="   + snapshot.get75thPercentile() +
+          ", 95th percentile age="   + snapshot.get95thPercentile() +
+          ", 98th percentile age="   + snapshot.get98thPercentile() +
+          ", 99th percentile age="   + snapshot.get99thPercentile() +
+          ", 99.9th percentile age=" + snapshot.get99thPercentile();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
index 50e8bbb..c208388 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
@@ -22,20 +22,15 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.util.Counter;
+import org.apache.hadoop.hbase.util.FastLongHistogram;
 
-import static com.codahale.metrics.MetricRegistry.name;
 
 /**
  * Class that implements cache metrics.
  */
 @InterfaceAudience.Private
 public class CacheStats {
-  /**
-   * Needed making histograms.
-   */
-  private static final MetricRegistry METRICS = new MetricRegistry();
 
   /** Sliding window statistics. The number of metric periods to include in
    * sliding window hit ratio calculations.
@@ -43,10 +38,10 @@ public class CacheStats {
   static final int DEFAULT_WINDOW_PERIODS = 5;
 
   /** The number of getBlock requests that were cache hits */
-  private final AtomicLong hitCount = new AtomicLong(0);
+  private final Counter hitCount = new Counter();
 
   /** The number of getBlock requests that were cache hits from primary replica */
-  private final AtomicLong primaryHitCount = new AtomicLong(0);
+  private final Counter primaryHitCount = new Counter();
   
   /**
    * The number of getBlock requests that were cache hits, but only from
@@ -54,27 +49,27 @@ public class CacheStats {
    * attempt to read from the block cache even if they will not put new blocks
    * into the block cache.  See HBASE-2253 for more information.
    */
-  private final AtomicLong hitCachingCount = new AtomicLong(0);
+  private final Counter hitCachingCount = new Counter();
 
   /** The number of getBlock requests that were cache misses */
-  private final AtomicLong missCount = new AtomicLong(0);
+  private final Counter missCount = new Counter();
 
   /** The number of getBlock requests for primary replica that were cache misses */
-  private final AtomicLong primaryMissCount = new AtomicLong(0);
+  private final Counter primaryMissCount = new Counter();
   /**
    * The number of getBlock requests that were cache misses, but only from
    * requests that were set to use the block cache.
    */
-  private final AtomicLong missCachingCount = new AtomicLong(0);
+  private final Counter missCachingCount = new Counter();
 
   /** The number of times an eviction has occurred */
-  private final AtomicLong evictionCount = new AtomicLong(0);
+  private final Counter evictionCount = new Counter();
 
   /** The total number of blocks that have been evicted */
-  private final AtomicLong evictedBlockCount = new AtomicLong(0);
+  private final Counter evictedBlockCount = new Counter();
 
   /** The total number of blocks for primary replica that have been evicted */
-  private final AtomicLong primaryEvictedBlockCount = new AtomicLong(0);
+  private final Counter primaryEvictedBlockCount = new Counter();
 
   /** The total number of blocks that were not inserted. */
   private final AtomicLong failedInserts = new AtomicLong(0);
@@ -102,7 +97,7 @@ public class CacheStats {
   /**
    * Keep running age at eviction time
    */
-  private Histogram ageAtEviction;
+  private FastLongHistogram ageAtEviction;
   private long startTime = System.nanoTime();
 
   public CacheStats(final String name) {
@@ -115,7 +110,7 @@ public class CacheStats {
     this.hitCachingCounts = initializeZeros(numPeriodsInWindow);
     this.requestCounts = initializeZeros(numPeriodsInWindow);
     this.requestCachingCounts = initializeZeros(numPeriodsInWindow);
-    this.ageAtEviction = METRICS.histogram(name(CacheStats.class, name + ".ageAtEviction"));
+    this.ageAtEviction = new FastLongHistogram();
   }
 
   @Override
@@ -127,14 +122,13 @@ public class CacheStats {
       ", evictedBlockCount=" + getEvictedCount() +
       ", primaryMissCount=" + getPrimaryMissCount() +
       ", primaryHitCount=" + getPrimaryHitCount() +
-      ", evictedAgeMean=" + snapshot.getMean() +
-      ", evictedAgeStdDev=" + snapshot.getStdDev();
+      ", evictedAgeMean=" + snapshot.getMean();
   }
 
   public void miss(boolean caching, boolean primary) {
-    missCount.incrementAndGet();
-    if (primary) primaryMissCount.incrementAndGet();
-    if (caching) missCachingCount.incrementAndGet();
+    missCount.increment();
+    if (primary) primaryMissCount.increment();
+    if (caching) missCachingCount.increment();
   }
 
   public void hit(boolean caching) {
@@ -142,20 +136,20 @@ public class CacheStats {
   }
 
   public void hit(boolean caching, boolean primary) {
-    hitCount.incrementAndGet();
-    if (primary) primaryHitCount.incrementAndGet();
-    if (caching) hitCachingCount.incrementAndGet();
+    hitCount.increment();
+    if (primary) primaryHitCount.increment();
+    if (caching) hitCachingCount.increment();
   }
 
   public void evict() {
-    evictionCount.incrementAndGet();
+    evictionCount.increment();
   }
 
   public void evicted(final long t, boolean primary) {
-    if (t > this.startTime) this.ageAtEviction.update(t - this.startTime);
-    this.evictedBlockCount.incrementAndGet();
+    if (t > this.startTime) this.ageAtEviction.add(t - this.startTime,1);
+    this.evictedBlockCount.increment();
     if (primary) {
-      primaryEvictedBlockCount.incrementAndGet();
+      primaryEvictedBlockCount.increment();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 1e1835f..e9fa05c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -34,7 +34,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.commons.logging.Log;
@@ -61,6 +60,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Counter;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.io.Writable;
 
@@ -179,17 +179,19 @@ public class HFile {
    */
   public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
   // For measuring number of checksum failures
-  static final AtomicLong checksumFailures = new AtomicLong();
+  static final Counter checksumFailures = new Counter();
 
   // for test purpose
-  public static final AtomicLong dataBlockReadCnt = new AtomicLong(0);
+  public static final Counter dataBlockReadCnt = new Counter();
 
   /**
    * Number of checksum verification failures. It also
    * clears the counter.
    */
   public static final long getChecksumFailuresCount() {
-    return checksumFailures.getAndSet(0);
+    long count = checksumFailures.get();
+    checksumFailures.set(0);
+    return count;
   }
 
   /** API required to write an {@link HFile} */

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index e7a1e5e..e2f524c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -1522,7 +1522,7 @@ public class HFileBlock implements Cacheable {
           HFile.LOG.warn(msg);
           throw new IOException(msg); // cannot happen case here
         }
-        HFile.checksumFailures.incrementAndGet(); // update metrics
+        HFile.checksumFailures.increment(); // update metrics
 
         // If we have a checksum failure, we fall back into a mode where
         // the next few reads use HDFS level checksums. We aim to make the

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index b2f5ded..239c63d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -1497,7 +1497,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
             assert cachedBlock.isUnpacked() : "Packed block leak.";
             if (cachedBlock.getBlockType().isData()) {
               if (updateCacheMetrics) {
-                HFile.dataBlockReadCnt.incrementAndGet();
+                HFile.dataBlockReadCnt.increment();
               }
               // Validate encoding type for data blocks. We include encoding
               // type in the cache key, and we expect it to match on a cache hit.
@@ -1537,7 +1537,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         }
 
         if (updateCacheMetrics && hfileBlock.getBlockType().isData()) {
-          HFile.dataBlockReadCnt.incrementAndGet();
+          HFile.dataBlockReadCnt.increment();
         }
 
         return unpacked;

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
index 51e6268..ec1d4d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
@@ -18,10 +18,9 @@
  */
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
+import org.apache.hadoop.hbase.util.Counter;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
@@ -29,8 +28,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  */
 @InterfaceAudience.Private
 public class BucketCacheStats extends CacheStats {
-  private final AtomicLong ioHitCount = new AtomicLong(0);
-  private final AtomicLong ioHitTime = new AtomicLong(0);
+  private final Counter ioHitCount = new Counter(0);
+  private final Counter ioHitTime = new Counter(0);
   private final static int nanoTime = 1000000;
   private long lastLogTime = EnvironmentEdgeManager.currentTime();
 
@@ -45,8 +44,8 @@ public class BucketCacheStats extends CacheStats {
   }
 
   public void ioHit(long time) {
-    ioHitCount.incrementAndGet();
-    ioHitTime.addAndGet(time);
+    ioHitCount.increment();
+    ioHitTime.add(time);
   }
 
   public long getIOHitsPerSecond() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
index d752e17..924e7f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
+import org.apache.hadoop.hbase.util.Counter;
 
 /**
  * KeyValueScanner adaptor over the Reader.  It also provides hooks into
@@ -60,7 +60,7 @@ public class StoreFileScanner implements KeyValueScanner {
   // if have encountered the next row. Only used for reversed scan
   private boolean stopSkippingKVsIfNextRow = false;
 
-  private static AtomicLong seekCount;
+  private static Counter seekCount;
 
   private ScanQueryMatcher matcher;
 
@@ -164,7 +164,7 @@ public class StoreFileScanner implements KeyValueScanner {
   }
 
   public boolean seek(Cell key) throws IOException {
-    if (seekCount != null) seekCount.incrementAndGet();
+    if (seekCount != null) seekCount.increment();
 
     try {
       try {
@@ -191,7 +191,7 @@ public class StoreFileScanner implements KeyValueScanner {
   }
 
   public boolean reseek(Cell key) throws IOException {
-    if (seekCount != null) seekCount.incrementAndGet();
+    if (seekCount != null) seekCount.increment();
 
     try {
       try {
@@ -424,7 +424,7 @@ public class StoreFileScanner implements KeyValueScanner {
     return seekCount.get();
   }
   static final void instrument() {
-    seekCount = new AtomicLong();
+    seekCount = new Counter();
   }
 
   @Override
@@ -447,7 +447,7 @@ public class StoreFileScanner implements KeyValueScanner {
         Cell key = originalKey;
         do {
           Cell seekKey = CellUtil.createFirstOnRow(key);
-          if (seekCount != null) seekCount.incrementAndGet();
+          if (seekCount != null) seekCount.increment();
           if (!hfs.seekBefore(seekKey)) {
             this.cur = null;
             return false;
@@ -455,7 +455,7 @@ public class StoreFileScanner implements KeyValueScanner {
           Cell curCell = hfs.getCell();
           Cell firstKeyOfPreviousRow = CellUtil.createFirstOnRow(curCell);
 
-          if (seekCount != null) seekCount.incrementAndGet();
+          if (seekCount != null) seekCount.increment();
           if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
             this.cur = null;
             return false;


[16/37] hbase git commit: HBASE-15222 Use less contended classes for metrics

Posted by sy...@apache.org.
HBASE-15222 Use less contended classes for metrics

Summary:
Use less contended things for metrics.
For histogram which was the largest culprit we use FastLongHistogram
For atomic long where possible we now use counter.

Test Plan: unit tests

Reviewers:

Subscribers:

Differential Revision: https://reviews.facebook.net/D54381


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

Branch: refs/heads/hbase-12439
Commit: 630a65825ed9a9c00f72bbfcac0588e1ab0cdd72
Parents: 20e14f4
Author: Elliott Clark <ec...@apache.org>
Authored: Thu Feb 18 09:54:05 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Feb 24 14:34:05 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/FastLongHistogram.java    | 162 ++++++++++++++-----
 .../hbase/util/TestFastLongHistogram.java       |  32 ++++
 .../apache/hadoop/hbase/metrics/BaseSource.java |   8 -
 .../apache/hadoop/metrics2/MetricHistogram.java |   3 +
 hbase-hadoop2-compat/pom.xml                    |   8 +-
 .../hbase/ipc/MetricsHBaseServerSourceImpl.java |  50 +++---
 .../MetricsAssignmentManagerSourceImpl.java     |  10 +-
 .../MetricsMasterFilesystemSourceImpl.java      |  14 +-
 .../hbase/master/MetricsMasterSourceImpl.java   |   4 +-
 .../hbase/master/MetricsSnapshotSourceImpl.java |   8 +-
 .../balancer/MetricsBalancerSourceImpl.java     |   8 +-
 .../hadoop/hbase/metrics/BaseSourceImpl.java    |  17 +-
 .../MetricsRegionServerSourceImpl.java          |  16 +-
 .../regionserver/MetricsRegionSourceImpl.java   |  24 +--
 .../regionserver/wal/MetricsWALSourceImpl.java  |  10 +-
 .../MetricsReplicationGlobalSourceSource.java   |  36 ++---
 .../MetricsReplicationSinkSourceImpl.java       |  16 +-
 .../MetricsReplicationSourceSourceImpl.java     |  36 ++---
 .../hbase/rest/MetricsRESTSourceImpl.java       |  38 ++---
 .../thrift/MetricsThriftServerSourceImpl.java   |  13 +-
 .../metrics2/lib/DynamicMetricsRegistry.java    | 103 ++----------
 .../metrics2/lib/MetricMutableQuantiles.java    | 154 ------------------
 .../metrics2/lib/MetricsExecutorImpl.java       |   2 +-
 .../hadoop/metrics2/lib/MutableFastCounter.java |  60 +++++++
 .../hadoop/metrics2/lib/MutableHistogram.java   | 133 +++++----------
 .../metrics2/lib/MutableRangeHistogram.java     |  75 ++++-----
 .../metrics2/lib/MutableSizeHistogram.java      |  25 ++-
 .../metrics2/lib/MutableTimeHistogram.java      |  23 ++-
 .../hbase/metrics/TestBaseSourceImpl.java       |   5 +-
 .../tmpl/regionserver/BlockCacheTmpl.jamon      |   8 -
 .../tmpl/regionserver/BlockCacheViewTmpl.jamon  |   1 -
 .../tmpl/regionserver/ServerMetricsTmpl.jamon   |   1 -
 .../hadoop/hbase/io/hfile/AgeSnapshot.java      |  38 +++--
 .../hadoop/hbase/io/hfile/BlockCacheUtil.java   |  31 ++--
 .../hadoop/hbase/io/hfile/CacheStats.java       |  54 +++----
 .../org/apache/hadoop/hbase/io/hfile/HFile.java |  10 +-
 .../hadoop/hbase/io/hfile/HFileBlock.java       |   2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |   4 +-
 .../hbase/io/hfile/bucket/BucketCacheStats.java |  11 +-
 .../hbase/regionserver/StoreFileScanner.java    |  14 +-
 40 files changed, 565 insertions(+), 702 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
index 623cbdb..78b2bf0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.util;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicLongArray;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -31,11 +30,20 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class FastLongHistogram {
+
+  /**
+   * Default number of bins.
+   */
+  public static final int DEFAULT_NBINS = 255;
+
+  public static final double[] DEFAULT_QUANTILES =
+      new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
+
   /**
    * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
    */
   private static class Bins {
-    private final AtomicLongArray counts;
+    private final Counter[] counts;
     // inclusive
     private final long binsMin;
     // exclusive
@@ -43,6 +51,10 @@ public class FastLongHistogram {
     private final long bins10XMax;
     private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
     private final AtomicLong max = new AtomicLong(0L);
+
+    private final Counter count = new Counter(0);
+    private final Counter total = new Counter(0);
+
     // set to true when any of data has been inserted to the Bins. It is set after the counts are
     // updated.
     private final AtomicBoolean hasData = new AtomicBoolean(false);
@@ -50,17 +62,18 @@ public class FastLongHistogram {
     /**
      * The constructor for creating a Bins without any prior data.
      */
-    public Bins() {
-      this.counts = new AtomicLongArray(4);
-      this.binsMin = 0L;
-      this.binsMax = Long.MAX_VALUE;
-      this.bins10XMax = Long.MAX_VALUE;
+    public Bins(int numBins) {
+      counts = createCounters(numBins + 3);
+      this.binsMin = 1L;
+
+      // These two numbers are total guesses
+      // and should be treated as highly suspect.
+      this.binsMax = 1000;
+      this.bins10XMax = binsMax * 10;
     }
-    
+
     /**
      * The constructor for creating a Bins with last Bins.
-     * @param last the last Bins instance.
-     * @param quantiles the quantiles for creating the bins of the histogram.
      */
     public Bins(Bins last, int numOfBins, double minQ, double maxQ) {
       long[] values = last.getQuantiles(new double[] { minQ, maxQ });
@@ -72,28 +85,51 @@ public class FastLongHistogram {
       this.binsMax = Math.max(binsMax, this.binsMin + numOfBins);
       this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
 
-      this.counts = new AtomicLongArray(numOfBins + 3);
+      this.counts = createCounters(numOfBins + 3);
+    }
+
+    private Counter[] createCounters(int num) {
+      Counter[] counters = new Counter[num];
+      for (int i = 0; i < num; i++) {
+        counters[i] = new Counter();
+      }
+      return counters;
+    }
+
+    private int getIndex(long value) {
+      if (value < this.binsMin) {
+        return 0;
+      } else if (value > this.bins10XMax) {
+        return this.counts.length - 1;
+      } else if (value >= this.binsMax) {
+        return this.counts.length - 2;
+      }
+      // compute the position
+      return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
+          (this.binsMax - this.binsMin));
+
     }
 
     /**
      * Adds a value to the histogram.
      */
     public void add(long value, long count) {
+      if (value < 0) {
+        // The whole computation is completely thrown off if there are negative numbers
+        //
+        // Normally we would throw an IllegalArgumentException however this is the metrics
+        // system and it should be completely safe at all times.
+        // So silently throw it away.
+        return;
+      }
       AtomicUtils.updateMin(min, value);
       AtomicUtils.updateMax(max, value);
 
-      if (value < this.binsMin) {
-        this.counts.addAndGet(0, count);
-      } else if (value > this.bins10XMax) {
-        this.counts.addAndGet(this.counts.length() - 1, count);
-      } else if (value >= this.binsMax) {
-        this.counts.addAndGet(this.counts.length() - 2, count);
-      } else {
-        // compute the position
-        int pos =
-            1 + (int) ((value - this.binsMin) * (this.counts.length() - 3) / (this.binsMax - this.binsMin));
-        this.counts.addAndGet(pos, count);
-      }
+      this.count.add(count);
+      this.total.add(value * count);
+
+      int pos = getIndex(value);
+      this.counts[pos].add(count);
 
       // hasData needs to be updated as last
       this.hasData.set(true);
@@ -101,7 +137,6 @@ public class FastLongHistogram {
     
     /**
      * Computes the quantiles give the ratios.
-     * @param smooth set to true to have a prior on the distribution. Used for recreating the bins.
      */
     public long[] getQuantiles(double[] quantiles) {
       if (!this.hasData.get()) {
@@ -112,10 +147,10 @@ public class FastLongHistogram {
       // Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
       // This is not synchronized, but since the counter are accumulating, the result is a good
       // estimation of a snapshot.
-      long[] counts = new long[this.counts.length()];
+      long[] counts = new long[this.counts.length];
       long total = 0L;
-      for (int i = 0; i < this.counts.length(); i++) {
-        counts[i] = this.counts.get(i);
+      for (int i = 0; i < this.counts.length; i++) {
+        counts[i] = this.counts[i].get();
         total += counts[i];
       }
 
@@ -137,8 +172,8 @@ public class FastLongHistogram {
           mn = this.binsMax;
           mx = this.bins10XMax;
         } else {
-          mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length() - 3);
-          mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length() - 3);
+          mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
+          mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
         }
 
         if (mx < this.min.get()) {
@@ -177,12 +212,27 @@ public class FastLongHistogram {
 
       return res;
     }
+
+
+    long getNumAtOrBelow(long val) {
+      final int targetIndex = getIndex(val);
+      long totalToCurrentIndex = 0;
+      for (int i = 0; i <= targetIndex; i++) {
+        totalToCurrentIndex += this.counts[i].get();
+      }
+      return  totalToCurrentIndex;
+    }
   }
 
   // The bins counting values. It is replaced with a new one in calling of reset().
-  private volatile Bins bins = new Bins();
-  // The quantiles for creating a Bins with last Bins.
-  private final int numOfBins;
+  private volatile Bins bins;
+
+  /**
+   * Constructor.
+   */
+  public FastLongHistogram() {
+    this(DEFAULT_NBINS);
+  }
 
   /**
    * Constructor.
@@ -190,7 +240,7 @@ public class FastLongHistogram {
    *          results but with lower efficiency, and vice versus.
    */
   public FastLongHistogram(int numOfBins) {
-    this.numOfBins = numOfBins;
+    this.bins = new Bins(numOfBins);
   }
 
   /**
@@ -202,10 +252,14 @@ public class FastLongHistogram {
    */
   public FastLongHistogram(int numOfBins, long min, long max) {
     this(numOfBins);
-    Bins bins = new Bins();
+    Bins bins = new Bins(numOfBins);
     bins.add(min, 1);
     bins.add(max, 1);
-    this.bins = new Bins(bins, numOfBins, 0.01, 0.99);
+    this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
+  }
+
+  private FastLongHistogram(Bins bins) {
+    this.bins = bins;
   }
 
   /**
@@ -222,12 +276,46 @@ public class FastLongHistogram {
     return this.bins.getQuantiles(quantiles);
   }
 
+  public long[] getQuantiles() {
+    return this.bins.getQuantiles(DEFAULT_QUANTILES);
+  }
+
+  public long getMin() {
+    return this.bins.min.get();
+  }
+
+  public long getMax() {
+    return this.bins.max.get();
+  }
+
+  public long getCount() {
+    return this.bins.count.get();
+  }
+
+  public long getMean() {
+    Bins bins = this.bins;
+    long count = bins.count.get();
+    long total = bins.total.get();
+    if (count == 0) {
+      return 0;
+    }
+    return total / count;
+  }
+
+  public long getNumAtOrBelow(long value) {
+    return this.bins.getNumAtOrBelow(value);
+  }
+
   /**
    * Resets the histogram for new counting.
    */
-  public void reset() {
+  public FastLongHistogram reset() {
     if (this.bins.hasData.get()) {
-      this.bins = new Bins(this.bins, numOfBins, 0.01, 0.99);
+      Bins oldBins = this.bins;
+      this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
+      return new FastLongHistogram(oldBins);
     }
+
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
index f5848f3..d56d143 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
@@ -26,6 +26,8 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.junit.Assert.assertEquals;
+
 /**
  * Testcases for FastLongHistogram.
  */
@@ -88,6 +90,36 @@ public class TestFastLongHistogram {
     }
   }
 
+
+  @Test
+  public void testGetNumAtOrBelow() {
+    long[] VALUES = { 1, 10, 20, 30, 40, 50 };
+
+    FastLongHistogram h = new FastLongHistogram();
+    for (long v : VALUES) {
+      for (int i = 0; i < 100; i++) {
+        h.add(v, 1);
+      }
+    }
+
+    h.add(Integer.MAX_VALUE, 1);
+
+    h.reset();
+
+    for (long v : VALUES) {
+      for (int i = 0; i < 100; i++) {
+        h.add(v, 1);
+      }
+    }
+    // Add something way out there to make sure it doesn't throw off the counts.
+    h.add(Integer.MAX_VALUE, 1);
+
+    assertEquals(100, h.getNumAtOrBelow(1));
+    assertEquals(200, h.getNumAtOrBelow(11));
+    assertEquals(601, h.getNumAtOrBelow(Long.MAX_VALUE));
+  }
+
+
   @Test
   public void testSameValues() {
     FastLongHistogram hist = new FastLongHistogram(100);

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
index 3ab783a..f79aa9f 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
@@ -80,14 +80,6 @@ public interface BaseSource {
 
 
   /**
-   * Add some value to a Quantile (An accurate histogram).
-   *
-   * @param name the name of the quantile
-   * @param value the value to add to the quantile
-   */
-  void updateQuantile(String name, long value);
-
-  /**
    * Get the metrics context.  For hadoop metrics2 system this is usually an all lowercased string.
    * eg. regionserver, master, thriftserver
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/MetricHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/MetricHistogram.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/MetricHistogram.java
index b759efb..9d24a23 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/MetricHistogram.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/MetricHistogram.java
@@ -30,10 +30,13 @@ public interface MetricHistogram {
   String MAX_METRIC_NAME = "_max";
   String MEAN_METRIC_NAME = "_mean";
   String MEDIAN_METRIC_NAME = "_median";
+  String TWENTY_FIFTH_PERCENTILE_METRIC_NAME = "_25th_percentile";
   String SEVENTY_FIFTH_PERCENTILE_METRIC_NAME = "_75th_percentile";
   String NINETIETH_PERCENTILE_METRIC_NAME = "_90th_percentile";
   String NINETY_FIFTH_PERCENTILE_METRIC_NAME = "_95th_percentile";
+  String NINETY_EIGHTH_PERCENTILE_METRIC_NAME = "_98th_percentile";
   String NINETY_NINETH_PERCENTILE_METRIC_NAME = "_99th_percentile";
+  String NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME = "_99.9th_percentile";
 
   /**
    * Add a single value to a histogram's stream of values.

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/pom.xml b/hbase-hadoop2-compat/pom.xml
index 100a297..56c3e7f 100644
--- a/hbase-hadoop2-compat/pom.xml
+++ b/hbase-hadoop2-compat/pom.xml
@@ -162,6 +162,10 @@ limitations under the License.
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>
@@ -182,10 +186,6 @@ limitations under the License.
       <version>${hadoop-two.version}</version>
     </dependency>
     <dependency>
-      <groupId>io.dropwizard.metrics</groupId>
-      <artifactId>metrics-core</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
index 487f9f5..48f57e9 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
@@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.ipc;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.Interns;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
-import org.apache.hadoop.metrics2.lib.MutableHistogram;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 
 @InterfaceAudience.Private
 public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
@@ -33,29 +33,29 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
 
 
   private final MetricsHBaseServerWrapper wrapper;
-  private final MutableCounterLong authorizationSuccesses;
-  private final MutableCounterLong authorizationFailures;
-  private final MutableCounterLong authenticationSuccesses;
-  private final MutableCounterLong authenticationFailures;
-  private final MutableCounterLong authenticationFallbacks;
-  private final MutableCounterLong sentBytes;
-  private final MutableCounterLong receivedBytes;
-
-  private final MutableCounterLong exceptions;
-  private final MutableCounterLong exceptionsOOO;
-  private final MutableCounterLong exceptionsBusy;
-  private final MutableCounterLong exceptionsUnknown;
-  private final MutableCounterLong exceptionsSanity;
-  private final MutableCounterLong exceptionsNSRE;
-  private final MutableCounterLong exceptionsMoved;
-  private final MutableCounterLong exceptionsMultiTooLarge;
-
-
-  private MutableHistogram queueCallTime;
-  private MutableHistogram processCallTime;
-  private MutableHistogram totalCallTime;
-  private MutableHistogram requestSize;
-  private MutableHistogram responseSize;
+  private final MutableFastCounter authorizationSuccesses;
+  private final MutableFastCounter authorizationFailures;
+  private final MutableFastCounter authenticationSuccesses;
+  private final MutableFastCounter authenticationFailures;
+  private final MutableFastCounter authenticationFallbacks;
+  private final MutableFastCounter sentBytes;
+  private final MutableFastCounter receivedBytes;
+
+  private final MutableFastCounter exceptions;
+  private final MutableFastCounter exceptionsOOO;
+  private final MutableFastCounter exceptionsBusy;
+  private final MutableFastCounter exceptionsUnknown;
+  private final MutableFastCounter exceptionsSanity;
+  private final MutableFastCounter exceptionsNSRE;
+  private final MutableFastCounter exceptionsMoved;
+  private final MutableFastCounter exceptionsMultiTooLarge;
+
+
+  private MetricHistogram queueCallTime;
+  private MetricHistogram processCallTime;
+  private MetricHistogram totalCallTime;
+  private MetricHistogram requestSize;
+  private MetricHistogram responseSize;
 
   public MetricsHBaseServerSourceImpl(String metricsName,
                                       String metricsDescription,

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
index ccf1c1d..a2192d2 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
@@ -20,17 +20,19 @@ package org.apache.hadoop.hbase.master;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
-import org.apache.hadoop.metrics2.lib.MutableHistogram;
 
 @InterfaceAudience.Private
-public class MetricsAssignmentManagerSourceImpl extends BaseSourceImpl implements MetricsAssignmentManagerSource {
+public class MetricsAssignmentManagerSourceImpl
+    extends BaseSourceImpl
+    implements MetricsAssignmentManagerSource {
 
   private MutableGaugeLong ritGauge;
   private MutableGaugeLong ritCountOverThresholdGauge;
   private MutableGaugeLong ritOldestAgeGauge;
-  private MutableHistogram assignTimeHisto;
-  private MutableHistogram bulkAssignTimeHisto;
+  private MetricHistogram assignTimeHisto;
+  private MetricHistogram bulkAssignTimeHisto;
 
   public MetricsAssignmentManagerSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
index 28414ea..7e8542c 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
@@ -20,15 +20,17 @@ package org.apache.hadoop.hbase.master;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
-import org.apache.hadoop.metrics2.lib.MutableHistogram;
+import org.apache.hadoop.metrics2.MetricHistogram;
 
 @InterfaceAudience.Private
-public class MetricsMasterFilesystemSourceImpl extends BaseSourceImpl implements MetricsMasterFileSystemSource {
+public class MetricsMasterFilesystemSourceImpl
+    extends BaseSourceImpl
+    implements MetricsMasterFileSystemSource {
 
-  private MutableHistogram splitSizeHisto;
-  private MutableHistogram splitTimeHisto;
-  private MutableHistogram metaSplitTimeHisto;
-  private MutableHistogram metaSplitSizeHisto;
+  private MetricHistogram splitSizeHisto;
+  private MetricHistogram splitTimeHisto;
+  private MetricHistogram metaSplitTimeHisto;
+  private MetricHistogram metaSplitSizeHisto;
 
   public MetricsMasterFilesystemSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
index b0ba66e..ea21967 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.Interns;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 
 /**
  * Hadoop2 implementation of MetricsMasterSource.
@@ -35,7 +35,7 @@ public class MetricsMasterSourceImpl
     extends BaseSourceImpl implements MetricsMasterSource {
 
   private final MetricsMasterWrapper masterWrapper;
-  private MutableCounterLong clusterRequestsCounter;
+  private MutableFastCounter clusterRequestsCounter;
 
   public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
     this(METRICS_NAME,

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
index c2fc6b9..bcefda0 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
@@ -20,14 +20,14 @@ package org.apache.hadoop.hbase.master;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
-import org.apache.hadoop.metrics2.lib.MutableHistogram;
+import org.apache.hadoop.metrics2.MetricHistogram;
 
 @InterfaceAudience.Private
 public class MetricsSnapshotSourceImpl extends BaseSourceImpl implements MetricsSnapshotSource {
 
-  private MutableHistogram snapshotTimeHisto;
-  private MutableHistogram snapshotCloneTimeHisto;
-  private MutableHistogram snapshotRestoreTimeHisto;
+  private MetricHistogram snapshotTimeHisto;
+  private MetricHistogram snapshotCloneTimeHisto;
+  private MetricHistogram snapshotRestoreTimeHisto;
 
   public MetricsSnapshotSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
index da34df2..0a74630 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
@@ -20,14 +20,14 @@ package org.apache.hadoop.hbase.master.balancer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
-import org.apache.hadoop.metrics2.lib.MutableHistogram;
+import org.apache.hadoop.metrics2.MetricHistogram;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 
 @InterfaceAudience.Private
 public class MetricsBalancerSourceImpl extends BaseSourceImpl implements MetricsBalancerSource{
 
-  private MutableHistogram blanceClusterHisto;
-  private MutableCounterLong miscCount;
+  private MetricHistogram blanceClusterHisto;
+  private MutableFastCounter miscCount;
 
   public MetricsBalancerSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
index 6756a21..f843ec2 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
@@ -24,8 +24,7 @@ import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
-import org.apache.hadoop.metrics2.lib.MetricMutableQuantiles;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableHistogram;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
@@ -88,7 +87,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
    * @param value     the new value of the gauge.
    */
   public void setGauge(String gaugeName, long value) {
-    MutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, value);
+    MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, value);
     gaugeInt.set(value);
   }
 
@@ -99,7 +98,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
    * @param delta     The amount to increment the gauge by.
    */
   public void incGauge(String gaugeName, long delta) {
-    MutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
+    MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0l);
     gaugeInt.incr(delta);
   }
 
@@ -110,7 +109,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
    * @param delta     the ammount to subtract from a gauge value.
    */
   public void decGauge(String gaugeName, long delta) {
-    MutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
+    MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0l);
     gaugeInt.decr(delta);
   }
 
@@ -121,7 +120,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
    * @param delta the ammount to increment
    */
   public void incCounters(String key, long delta) {
-    MutableCounterLong counter = metricsRegistry.getLongCounter(key, 0l);
+    MutableFastCounter counter = metricsRegistry.getCounter(key, 0l);
     counter.incr(delta);
 
   }
@@ -132,12 +131,6 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
     histo.add(value);
   }
 
-  @Override
-  public void updateQuantile(String name, long value) {
-    MetricMutableQuantiles histo = metricsRegistry.getQuantile(name);
-    histo.add(value);
-  }
-
   /**
    * Remove a named gauge.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index f669d26..4a2d8cc 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.Interns;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 
 /**
  * Hadoop2 implementation of MetricsRegionServerSource.
@@ -45,13 +45,13 @@ public class MetricsRegionServerSourceImpl
   private final MetricHistogram replayHisto;
   private final MetricHistogram scanNextHisto;
 
-  private final MutableCounterLong slowPut;
-  private final MutableCounterLong slowDelete;
-  private final MutableCounterLong slowGet;
-  private final MutableCounterLong slowIncrement;
-  private final MutableCounterLong slowAppend;
-  private final MutableCounterLong splitRequest;
-  private final MutableCounterLong splitSuccess;
+  private final MutableFastCounter slowPut;
+  private final MutableFastCounter slowDelete;
+  private final MutableFastCounter slowGet;
+  private final MutableFastCounter slowIncrement;
+  private final MutableFastCounter slowAppend;
+  private final MutableFastCounter splitRequest;
+  private final MutableFastCounter splitSuccess;
 
   private final MetricHistogram splitTimeHisto;
   private final MetricHistogram flushTimeHisto;

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
index 90c6ce2..c6b7d12 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
@@ -23,11 +23,11 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
 import org.apache.hadoop.metrics2.lib.Interns;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
-import org.apache.hadoop.metrics2.lib.MutableHistogram;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 
 @InterfaceAudience.Private
 public class MetricsRegionSourceImpl implements MetricsRegionSource {
@@ -53,12 +53,12 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
   private final String regionAppendKey;
   private final String regionScanNextKey;
 
-  private final MutableCounterLong regionPut;
-  private final MutableCounterLong regionDelete;
-  private final MutableCounterLong regionIncrement;
-  private final MutableCounterLong regionAppend;
-  private final MutableHistogram regionGet;
-  private final MutableHistogram regionScanNext;
+  private final MutableFastCounter regionPut;
+  private final MutableFastCounter regionDelete;
+  private final MutableFastCounter regionIncrement;
+  private final MutableFastCounter regionAppend;
+  private final MetricHistogram regionGet;
+  private final MetricHistogram regionScanNext;
   private final int hashCode;
 
   public MetricsRegionSourceImpl(MetricsRegionWrapper regionWrapper,
@@ -80,16 +80,16 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
     String suffix = "Count";
 
     regionPutKey = regionNamePrefix + MetricsRegionServerSource.MUTATE_KEY + suffix;
-    regionPut = registry.getLongCounter(regionPutKey, 0L);
+    regionPut = registry.getCounter(regionPutKey, 0L);
 
     regionDeleteKey = regionNamePrefix + MetricsRegionServerSource.DELETE_KEY + suffix;
-    regionDelete = registry.getLongCounter(regionDeleteKey, 0L);
+    regionDelete = registry.getCounter(regionDeleteKey, 0L);
 
     regionIncrementKey = regionNamePrefix + MetricsRegionServerSource.INCREMENT_KEY + suffix;
-    regionIncrement = registry.getLongCounter(regionIncrementKey, 0L);
+    regionIncrement = registry.getCounter(regionIncrementKey, 0L);
 
     regionAppendKey = regionNamePrefix + MetricsRegionServerSource.APPEND_KEY + suffix;
-    regionAppend = registry.getLongCounter(regionAppendKey, 0L);
+    regionAppend = registry.getCounter(regionAppendKey, 0L);
 
     regionGetKey = regionNamePrefix + MetricsRegionServerSource.GET_KEY;
     regionGet = registry.newTimeHistogram(regionGetKey);

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
index a149d1b..995e334 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
 import org.apache.hadoop.metrics2.MetricHistogram;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 
 
 /**
@@ -36,10 +36,10 @@ public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSo
   private final MetricHistogram appendSizeHisto;
   private final MetricHistogram appendTimeHisto;
   private final MetricHistogram syncTimeHisto;
-  private final MutableCounterLong appendCount;
-  private final MutableCounterLong slowAppendCount;
-  private final MutableCounterLong logRollRequested;
-  private final MutableCounterLong lowReplicationLogRollRequested;
+  private final MutableFastCounter appendCount;
+  private final MutableFastCounter slowAppendCount;
+  private final MutableFastCounter logRollRequested;
+  private final MutableFastCounter lowReplicationLogRollRequested;
 
   public MetricsWALSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
index 392cd39..93b10b6 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
 public class MetricsReplicationGlobalSourceSource implements MetricsReplicationSourceSource{
@@ -26,38 +26,38 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
 
   private final MutableGaugeLong ageOfLastShippedOpGauge;
   private final MutableGaugeLong sizeOfLogQueueGauge;
-  private final MutableCounterLong logReadInEditsCounter;
-  private final MutableCounterLong logEditsFilteredCounter;
-  private final MutableCounterLong shippedBatchesCounter;
-  private final MutableCounterLong shippedOpsCounter;
-  private final MutableCounterLong shippedKBsCounter;
-  private final MutableCounterLong logReadInBytesCounter;
-  private final MutableCounterLong shippedHFilesCounter;
+  private final MutableFastCounter logReadInEditsCounter;
+  private final MutableFastCounter logEditsFilteredCounter;
+  private final MutableFastCounter shippedBatchesCounter;
+  private final MutableFastCounter shippedOpsCounter;
+  private final MutableFastCounter shippedKBsCounter;
+  private final MutableFastCounter logReadInBytesCounter;
+  private final MutableFastCounter shippedHFilesCounter;
   private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
 
   public MetricsReplicationGlobalSourceSource(MetricsReplicationSourceImpl rms) {
     this.rms = rms;
 
-    ageOfLastShippedOpGauge = rms.getMetricsRegistry().getLongGauge(SOURCE_AGE_OF_LAST_SHIPPED_OP, 0L);
+    ageOfLastShippedOpGauge = rms.getMetricsRegistry().getGauge(SOURCE_AGE_OF_LAST_SHIPPED_OP, 0L);
 
-    sizeOfLogQueueGauge = rms.getMetricsRegistry().getLongGauge(SOURCE_SIZE_OF_LOG_QUEUE, 0L);
+    sizeOfLogQueueGauge = rms.getMetricsRegistry().getGauge(SOURCE_SIZE_OF_LOG_QUEUE, 0L);
 
-    shippedBatchesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_BATCHES, 0L);
+    shippedBatchesCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_BATCHES, 0L);
 
-    shippedOpsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_OPS, 0L);
+    shippedOpsCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_OPS, 0L);
 
-    shippedKBsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_KBS, 0L);
+    shippedKBsCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_KBS, 0L);
 
-    logReadInBytesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_READ_IN_BYTES, 0L);
+    logReadInBytesCounter = rms.getMetricsRegistry().getCounter(SOURCE_LOG_READ_IN_BYTES, 0L);
 
-    logReadInEditsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_READ_IN_EDITS, 0L);
+    logReadInEditsCounter = rms.getMetricsRegistry().getCounter(SOURCE_LOG_READ_IN_EDITS, 0L);
 
-    logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_EDITS_FILTERED, 0L);
+    logEditsFilteredCounter = rms.getMetricsRegistry().getCounter(SOURCE_LOG_EDITS_FILTERED, 0L);
 
-    shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_HFILES, 0L);
+    shippedHFilesCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_HFILES, 0L);
 
     sizeOfHFileRefsQueueGauge =
-        rms.getMetricsRegistry().getLongGauge(SOURCE_SIZE_OF_HFILE_REFS_QUEUE, 0L);
+        rms.getMetricsRegistry().getGauge(SOURCE_SIZE_OF_HFILE_REFS_QUEUE, 0L);
   }
 
   @Override public void setLastShippedAge(long age) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
index 8f4a337..540212a 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
@@ -18,21 +18,21 @@
 
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
 public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkSource {
 
   private final MutableGaugeLong ageGauge;
-  private final MutableCounterLong batchesCounter;
-  private final MutableCounterLong opsCounter;
-  private final MutableCounterLong hfilesCounter;
+  private final MutableFastCounter batchesCounter;
+  private final MutableFastCounter opsCounter;
+  private final MutableFastCounter hfilesCounter;
 
   public MetricsReplicationSinkSourceImpl(MetricsReplicationSourceImpl rms) {
-    ageGauge = rms.getMetricsRegistry().getLongGauge(SINK_AGE_OF_LAST_APPLIED_OP, 0L);
-    batchesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_BATCHES, 0L);
-    opsCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_OPS, 0L);
-    hfilesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_HFILES, 0L);
+    ageGauge = rms.getMetricsRegistry().getGauge(SINK_AGE_OF_LAST_APPLIED_OP, 0L);
+    batchesCounter = rms.getMetricsRegistry().getCounter(SINK_APPLIED_BATCHES, 0L);
+    opsCounter = rms.getMetricsRegistry().getCounter(SINK_APPLIED_OPS, 0L);
+    hfilesCounter = rms.getMetricsRegistry().getCounter(SINK_APPLIED_HFILES, 0L);
   }
 
   @Override public void setLastAppliedOpAge(long age) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
index 217cc3e..9941712 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
 public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSourceSource {
@@ -37,13 +37,13 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
 
   private final MutableGaugeLong ageOfLastShippedOpGauge;
   private final MutableGaugeLong sizeOfLogQueueGauge;
-  private final MutableCounterLong logReadInEditsCounter;
-  private final MutableCounterLong logEditsFilteredCounter;
-  private final MutableCounterLong shippedBatchesCounter;
-  private final MutableCounterLong shippedOpsCounter;
-  private final MutableCounterLong shippedKBsCounter;
-  private final MutableCounterLong logReadInBytesCounter;
-  private final MutableCounterLong shippedHFilesCounter;
+  private final MutableFastCounter logReadInEditsCounter;
+  private final MutableFastCounter logEditsFilteredCounter;
+  private final MutableFastCounter shippedBatchesCounter;
+  private final MutableFastCounter shippedOpsCounter;
+  private final MutableFastCounter shippedKBsCounter;
+  private final MutableFastCounter logReadInBytesCounter;
+  private final MutableFastCounter shippedHFilesCounter;
   private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
 
   public MetricsReplicationSourceSourceImpl(MetricsReplicationSourceImpl rms, String id) {
@@ -51,34 +51,34 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
     this.id = id;
 
     ageOfLastShippedOpKey = "source." + id + ".ageOfLastShippedOp";
-    ageOfLastShippedOpGauge = rms.getMetricsRegistry().getLongGauge(ageOfLastShippedOpKey, 0L);
+    ageOfLastShippedOpGauge = rms.getMetricsRegistry().getGauge(ageOfLastShippedOpKey, 0L);
 
     sizeOfLogQueueKey = "source." + id + ".sizeOfLogQueue";
-    sizeOfLogQueueGauge = rms.getMetricsRegistry().getLongGauge(sizeOfLogQueueKey, 0L);
+    sizeOfLogQueueGauge = rms.getMetricsRegistry().getGauge(sizeOfLogQueueKey, 0L);
 
     shippedBatchesKey = "source." + this.id + ".shippedBatches";
-    shippedBatchesCounter = rms.getMetricsRegistry().getLongCounter(shippedBatchesKey, 0L);
+    shippedBatchesCounter = rms.getMetricsRegistry().getCounter(shippedBatchesKey, 0L);
 
     shippedOpsKey = "source." + this.id + ".shippedOps";
-    shippedOpsCounter = rms.getMetricsRegistry().getLongCounter(shippedOpsKey, 0L);
+    shippedOpsCounter = rms.getMetricsRegistry().getCounter(shippedOpsKey, 0L);
 
     shippedKBsKey = "source." + this.id + ".shippedKBs";
-    shippedKBsCounter = rms.getMetricsRegistry().getLongCounter(shippedKBsKey, 0L);
+    shippedKBsCounter = rms.getMetricsRegistry().getCounter(shippedKBsKey, 0L);
 
     logReadInBytesKey = "source." + this.id + ".logReadInBytes";
-    logReadInBytesCounter = rms.getMetricsRegistry().getLongCounter(logReadInBytesKey, 0L);
+    logReadInBytesCounter = rms.getMetricsRegistry().getCounter(logReadInBytesKey, 0L);
 
     logReadInEditsKey = "source." + id + ".logEditsRead";
-    logReadInEditsCounter = rms.getMetricsRegistry().getLongCounter(logReadInEditsKey, 0L);
+    logReadInEditsCounter = rms.getMetricsRegistry().getCounter(logReadInEditsKey, 0L);
 
     logEditsFilteredKey = "source." + id + ".logEditsFiltered";
-    logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(logEditsFilteredKey, 0L);
+    logEditsFilteredCounter = rms.getMetricsRegistry().getCounter(logEditsFilteredKey, 0L);
 
     shippedHFilesKey = "source." + this.id + ".shippedHFiles";
-    shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(shippedHFilesKey, 0L);
+    shippedHFilesCounter = rms.getMetricsRegistry().getCounter(shippedHFilesKey, 0L);
 
     sizeOfHFileRefsQueueKey = "source." + id + ".sizeOfHFileRefsQueue";
-    sizeOfHFileRefsQueueGauge = rms.getMetricsRegistry().getLongGauge(sizeOfHFileRefsQueueKey, 0L);
+    sizeOfHFileRefsQueueGauge = rms.getMetricsRegistry().getGauge(sizeOfHFileRefsQueueKey, 0L);
   }
 
   @Override public void setLastShippedAge(long age) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java
index 9eae18b..71755b7 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/rest/MetricsRESTSourceImpl.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.rest;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 
 /**
  * Hadoop Two implementation of a metrics2 source that will export metrics from the Rest server to
@@ -31,15 +31,15 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 @InterfaceAudience.Private
 public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsRESTSource {
 
-  private MutableCounterLong request;
-  private MutableCounterLong sucGet;
-  private MutableCounterLong sucPut;
-  private MutableCounterLong sucDel;
-  private MutableCounterLong sucScan;
-  private MutableCounterLong fGet;
-  private MutableCounterLong fPut;
-  private MutableCounterLong fDel;
-  private MutableCounterLong fScan;
+  private MutableFastCounter request;
+  private MutableFastCounter sucGet;
+  private MutableFastCounter sucPut;
+  private MutableFastCounter sucDel;
+  private MutableFastCounter sucScan;
+  private MutableFastCounter fGet;
+  private MutableFastCounter fPut;
+  private MutableFastCounter fDel;
+  private MutableFastCounter fScan;
 
   public MetricsRESTSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, CONTEXT, JMX_CONTEXT);
@@ -55,17 +55,17 @@ public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsREST
   @Override
   public void init() {
     super.init();
-    request = getMetricsRegistry().getLongCounter(REQUEST_KEY, 0l);
+    request = getMetricsRegistry().getCounter(REQUEST_KEY, 0l);
 
-    sucGet = getMetricsRegistry().getLongCounter(SUCCESSFUL_GET_KEY, 0l);
-    sucPut = getMetricsRegistry().getLongCounter(SUCCESSFUL_PUT_KEY, 0l);
-    sucDel = getMetricsRegistry().getLongCounter(SUCCESSFUL_DELETE_KEY, 0l);
-    sucScan = getMetricsRegistry().getLongCounter(SUCCESSFUL_SCAN_KEY, 0L);
+    sucGet = getMetricsRegistry().getCounter(SUCCESSFUL_GET_KEY, 0l);
+    sucPut = getMetricsRegistry().getCounter(SUCCESSFUL_PUT_KEY, 0l);
+    sucDel = getMetricsRegistry().getCounter(SUCCESSFUL_DELETE_KEY, 0l);
+    sucScan = getMetricsRegistry().getCounter(SUCCESSFUL_SCAN_KEY, 0L);
 
-    fGet = getMetricsRegistry().getLongCounter(FAILED_GET_KEY, 0l);
-    fPut = getMetricsRegistry().getLongCounter(FAILED_PUT_KEY, 0l);
-    fDel = getMetricsRegistry().getLongCounter(FAILED_DELETE_KEY, 0l);
-    fScan = getMetricsRegistry().getLongCounter(FAILED_SCAN_KEY, 0l);
+    fGet = getMetricsRegistry().getCounter(FAILED_GET_KEY, 0l);
+    fPut = getMetricsRegistry().getCounter(FAILED_PUT_KEY, 0l);
+    fDel = getMetricsRegistry().getCounter(FAILED_DELETE_KEY, 0l);
+    fScan = getMetricsRegistry().getCounter(FAILED_SCAN_KEY, 0l);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
index f9612e5..f5d83b1 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.thrift;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableHistogram;
 
@@ -32,12 +33,12 @@ import org.apache.hadoop.metrics2.lib.MutableHistogram;
 public class MetricsThriftServerSourceImpl extends BaseSourceImpl implements
     MetricsThriftServerSource {
 
-  private MutableHistogram batchGetStat;
-  private MutableHistogram batchMutateStat;
-  private MutableHistogram queueTimeStat;
+  private MetricHistogram batchGetStat;
+  private MetricHistogram batchMutateStat;
+  private MetricHistogram queueTimeStat;
 
-  private MutableHistogram thriftCallStat;
-  private MutableHistogram thriftSlowCallStat;
+  private MetricHistogram thriftCallStat;
+  private MetricHistogram thriftSlowCallStat;
 
   private MutableGaugeLong callQueueLenGauge;
 
@@ -56,7 +57,7 @@ public class MetricsThriftServerSourceImpl extends BaseSourceImpl implements
     queueTimeStat = getMetricsRegistry().newTimeHistogram(TIME_IN_QUEUE_KEY);
     thriftCallStat = getMetricsRegistry().newTimeHistogram(THRIFT_CALL_KEY);
     thriftSlowCallStat = getMetricsRegistry().newTimeHistogram(SLOW_THRIFT_CALL_KEY);
-    callQueueLenGauge = getMetricsRegistry().getLongGauge(CALL_QUEUE_LEN_KEY, 0);
+    callQueueLenGauge = getMetricsRegistry().getGauge(CALL_QUEUE_LEN_KEY, 0);
 
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
index ee13c76..1f3da12 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
@@ -106,35 +106,13 @@ public class DynamicMetricsRegistry {
   }
 
   /**
-   * Create a mutable integer counter
-   * @param name  of the metric
-   * @param desc  metric description
-   * @param iVal  initial value
-   * @return a new counter object
-   */
-  public MutableCounterInt newCounter(String name, String desc, int iVal) {
-    return newCounter(new MetricsInfoImpl(name, desc), iVal);
-  }
-
-  /**
-   * Create a mutable integer counter
-   * @param info  metadata of the metric
-   * @param iVal  initial value
-   * @return a new counter object
-   */
-  public MutableCounterInt newCounter(MetricsInfo info, int iVal) {
-    MutableCounterInt ret = new MutableCounterInt(info, iVal);
-    return addNewMetricIfAbsent(info.name(), ret, MutableCounterInt.class);
-  }
-
-  /**
    * Create a mutable long integer counter
    * @param name  of the metric
    * @param desc  metric description
    * @param iVal  initial value
    * @return a new counter object
    */
-  public MutableCounterLong newCounter(String name, String desc, long iVal) {
+  public MutableFastCounter newCounter(String name, String desc, long iVal) {
     return newCounter(new MetricsInfoImpl(name, desc), iVal);
   }
 
@@ -144,30 +122,9 @@ public class DynamicMetricsRegistry {
    * @param iVal  initial value
    * @return a new counter object
    */
-  public MutableCounterLong newCounter(MetricsInfo info, long iVal) {
-    MutableCounterLong ret = new MutableCounterLong(info, iVal);
-    return addNewMetricIfAbsent(info.name(), ret, MutableCounterLong.class);
-  }
-
-  /**
-   * Create a mutable integer gauge
-   * @param name  of the metric
-   * @param desc  metric description
-   * @param iVal  initial value
-   * @return a new gauge object
-   */
-  public MutableGaugeInt newGauge(String name, String desc, int iVal) {
-    return newGauge(new MetricsInfoImpl(name, desc), iVal);
-  }
-  /**
-   * Create a mutable integer gauge
-   * @param info  metadata of the metric
-   * @param iVal  initial value
-   * @return a new gauge object
-   */
-  public MutableGaugeInt newGauge(MetricsInfo info, int iVal) {
-    MutableGaugeInt ret = new MutableGaugeInt(info, iVal);
-    return addNewMetricIfAbsent(info.name(), ret, MutableGaugeInt.class);
+  public MutableFastCounter newCounter(MetricsInfo info, long iVal) {
+    MutableFastCounter ret = new MutableFastCounter(info, iVal);
+    return addNewMetricIfAbsent(info.name(), ret, MutableFastCounter.class);
   }
 
   /**
@@ -326,19 +283,6 @@ public class DynamicMetricsRegistry {
     return addNewMetricIfAbsent(name, histo, MutableSizeHistogram.class);
   }
 
-  /**
-   * Create a new MutableQuantile(A more accurate histogram).
-   * @param name The name of the histogram
-   * @return a new MutableQuantile
-   */
-  public MetricMutableQuantiles newQuantile(String name) {
-    return newQuantile(name, "");
-  }
-
-  public MetricMutableQuantiles newQuantile(String name, String desc) {
-    MetricMutableQuantiles histo = new MetricMutableQuantiles(name, desc, "Ops", "", 60);
-    return addNewMetricIfAbsent(name, histo, MetricMutableQuantiles.class);
-  }
 
   synchronized void add(String name, MutableMetric metric) {
     addNewMetricIfAbsent(name, metric, MutableMetric.class);
@@ -475,7 +419,7 @@ public class DynamicMetricsRegistry {
    * @param gaugeName              name of the gauge to create or get.
    * @param potentialStartingValue value of the new gauge if we have to create it.
    */
-  public MutableGaugeLong getLongGauge(String gaugeName, long potentialStartingValue) {
+  public MutableGaugeLong getGauge(String gaugeName, long potentialStartingValue) {
     //Try and get the guage.
     MutableMetric metric = metricsMap.get(gaugeName);
 
@@ -510,12 +454,12 @@ public class DynamicMetricsRegistry {
    * @param counterName            Name of the counter to get
    * @param potentialStartingValue starting value if we have to create a new counter
    */
-  public MutableCounterLong getLongCounter(String counterName, long potentialStartingValue) {
-    //See getLongGauge for description on how this works.
+  public MutableFastCounter getCounter(String counterName, long potentialStartingValue) {
+    //See getGauge for description on how this works.
     MutableMetric counter = metricsMap.get(counterName);
     if (counter == null) {
-      MutableCounterLong newCounter =
-              new MutableCounterLong(new MetricsInfoImpl(counterName, ""), potentialStartingValue);
+      MutableFastCounter newCounter =
+              new MutableFastCounter(new MetricsInfoImpl(counterName, ""), potentialStartingValue);
       counter = metricsMap.putIfAbsent(counterName, newCounter);
       if (counter == null) {
         return newCounter;
@@ -523,16 +467,16 @@ public class DynamicMetricsRegistry {
     }
 
 
-    if (!(counter instanceof MutableCounterLong)) {
+    if (!(counter instanceof MutableCounter)) {
       throw new MetricsException("Metric already exists in registry for metric name: " +
-              counterName + " and not of type MetricMutableCounterLong");
+              counterName + " and not of type MutableCounter");
     }
 
-    return (MutableCounterLong) counter;
+    return (MutableFastCounter) counter;
   }
 
   public MutableHistogram getHistogram(String histoName) {
-    //See getLongGauge for description on how this works.
+    //See getGauge for description on how this works.
     MutableMetric histo = metricsMap.get(histoName);
     if (histo == null) {
       MutableHistogram newCounter =
@@ -552,27 +496,6 @@ public class DynamicMetricsRegistry {
     return (MutableHistogram) histo;
   }
 
-  public MetricMutableQuantiles getQuantile(String histoName) {
-    //See getLongGauge for description on how this works.
-    MutableMetric histo = metricsMap.get(histoName);
-    if (histo == null) {
-      MetricMutableQuantiles newCounter =
-          new MetricMutableQuantiles(histoName, "", "Ops", "", 60);
-      histo = metricsMap.putIfAbsent(histoName, newCounter);
-      if (histo == null) {
-        return newCounter;
-      }
-    }
-
-
-    if (!(histo instanceof MetricMutableQuantiles)) {
-      throw new MetricsException("Metric already exists in registry for metric name: " +
-          histoName + " and not of type MutableHistogram");
-    }
-
-    return (MetricMutableQuantiles) histo;
-  }
-
   private<T extends MutableMetric> T
   addNewMetricIfAbsent(String name,
                        T ret,

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java
deleted file mode 100644
index c03654b..0000000
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricMutableQuantiles.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.metrics2.lib;
-
-import static org.apache.hadoop.metrics2.lib.Interns.info;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.metrics2.MetricHistogram;
-import org.apache.hadoop.metrics2.MetricsExecutor;
-import org.apache.hadoop.metrics2.MetricsInfo;
-import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-import org.apache.hadoop.metrics2.util.MetricQuantile;
-import org.apache.hadoop.metrics2.util.MetricSampleQuantiles;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Watches a stream of long values, maintaining online estimates of specific quantiles with provably
- * low error bounds. This is particularly useful for accurate high-percentile (e.g. 95th, 99th)
- * latency metrics.
- */
-@InterfaceAudience.Private
-public class MetricMutableQuantiles extends MutableMetric implements MetricHistogram {
-
-  static final MetricQuantile[] quantiles = {new MetricQuantile(0.50, 0.050),
-      new MetricQuantile(0.75, 0.025), new MetricQuantile(0.90, 0.010),
-      new MetricQuantile(0.95, 0.005), new MetricQuantile(0.99, 0.001)};
-
-  private final MetricsInfo numInfo;
-  private final MetricsInfo[] quantileInfos;
-  private final int interval;
-
-  private MetricSampleQuantiles estimator;
-  private long previousCount = 0;
-  private MetricsExecutor executor;
-
-
-  @VisibleForTesting
-  protected Map<MetricQuantile, Long> previousSnapshot = null;
-
-  /**
-   * Instantiates a new {@link MetricMutableQuantiles} for a metric that rolls itself over on the
-   * specified time interval.
-   *
-   * @param name        of the metric
-   * @param description long-form textual description of the metric
-   * @param sampleName  type of items in the stream (e.g., "Ops")
-   * @param valueName   type of the values
-   * @param interval    rollover interval (in seconds) of the estimator
-   */
-  public MetricMutableQuantiles(String name, String description, String sampleName,
-                                String valueName, int interval) {
-    String ucName = StringUtils.capitalize(name);
-    String usName = StringUtils.capitalize(sampleName);
-    String uvName = StringUtils.capitalize(valueName);
-    String desc = StringUtils.uncapitalize(description);
-    String lsName = StringUtils.uncapitalize(sampleName);
-    String lvName = StringUtils.uncapitalize(valueName);
-
-    numInfo = info(ucName + "Num" + usName, String.format(
-        "Number of %s for %s with %ds interval", lsName, desc, interval));
-    // Construct the MetricsInfos for the quantiles, converting to percentiles
-    quantileInfos = new MetricsInfo[quantiles.length];
-    String nameTemplate = "%s%dthPercentile%dsInterval%s";
-    String descTemplate = "%d percentile %s with %d second interval for %s";
-    for (int i = 0; i < quantiles.length; i++) {
-      int percentile = (int) (100 * quantiles[i].quantile);
-      quantileInfos[i] = info(String.format(nameTemplate, ucName, percentile, interval, uvName),
-          String.format(descTemplate, percentile, lvName, interval, desc));
-    }
-
-    estimator = new MetricSampleQuantiles(quantiles);
-    executor = new MetricsExecutorImpl();
-    this.interval = interval;
-    executor.getExecutor().scheduleAtFixedRate(new RolloverSample(this),
-        interval,
-        interval,
-        TimeUnit.SECONDS);
-  }
-
-  @Override
-  public synchronized void snapshot(MetricsRecordBuilder builder, boolean all) {
-    if (all || changed()) {
-      builder.addGauge(numInfo, previousCount);
-      for (int i = 0; i < quantiles.length; i++) {
-        long newValue = 0;
-        // If snapshot is null, we failed to update since the window was empty
-        if (previousSnapshot != null) {
-          newValue = previousSnapshot.get(quantiles[i]);
-        }
-        builder.addGauge(quantileInfos[i], newValue);
-      }
-      if (changed()) {
-        clearChanged();
-      }
-    }
-  }
-
-  public synchronized void add(long value) {
-    estimator.insert(value);
-  }
-
-  public int getInterval() {
-    return interval;
-  }
-
-  /** Runnable used to periodically roll over the internal {@link org.apache.hadoop.metrics2.util.MetricSampleQuantiles} every interval. */
-  private static class RolloverSample implements Runnable {
-
-    MetricMutableQuantiles parent;
-
-    public RolloverSample(MetricMutableQuantiles parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    public void run() {
-      synchronized (parent) {
-        try {
-          parent.previousCount = parent.estimator.getCount();
-          parent.previousSnapshot = parent.estimator.snapshot();
-        } catch (IOException e) {
-          // Couldn't get a new snapshot because the window was empty
-          parent.previousCount = 0;
-          parent.previousSnapshot = null;
-        }
-        parent.estimator.clear();
-      }
-      parent.setChanged();
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java
index f70413e..c381609 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MetricsExecutorImpl.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.metrics2.MetricsExecutor;
 
 /**
  *  Class to handle the ScheduledExecutorService{@link ScheduledExecutorService} used by
- *  MetricMutableQuantiles{@link MetricMutableQuantiles}, MetricsRegionAggregateSourceImpl, and
+ *  MetricsRegionAggregateSourceImpl, and
  *  JmxCacheBuster
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableFastCounter.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableFastCounter.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableFastCounter.java
new file mode 100644
index 0000000..3f6c4a7
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableFastCounter.java
@@ -0,0 +1,60 @@
+/**
+ * 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.metrics2.lib;
+
+import org.apache.hadoop.hbase.util.Counter;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+
+public class MutableFastCounter extends MutableCounter {
+
+  private final Counter counter;
+
+  protected MutableFastCounter(MetricsInfo info, long iVal) {
+    super(info);
+    counter = new Counter(iVal);
+  }
+
+  @Override
+  public void incr() {
+    counter.increment();
+    setChanged();
+  }
+
+  /**
+   * Increment the value by a delta
+   * @param delta of the increment
+   */
+  public void incr(long delta) {
+    counter.add(delta);
+    setChanged();
+  }
+
+  @Override
+  public void snapshot(MetricsRecordBuilder builder, boolean all) {
+    if (all || changed()) {
+      builder.addCounter(info(), value());
+      clearChanged();
+    }
+  }
+
+  public long value() {
+    return counter.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
index c7ff940..717e0ee 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
@@ -18,124 +18,79 @@
 
 package org.apache.hadoop.metrics2.lib;
 
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Counter;
+import org.apache.hadoop.hbase.util.FastLongHistogram;
 import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 
-import com.codahale.metrics.ExponentiallyDecayingReservoir;
-import com.codahale.metrics.Reservoir;
-import com.codahale.metrics.Snapshot;
-
 /**
  * A histogram implementation that runs in constant space, and exports to hadoop2's metrics2 system.
  */
 @InterfaceAudience.Private
 public class MutableHistogram extends MutableMetric implements MetricHistogram {
-
-  private static final int DEFAULT_SAMPLE_SIZE = 2046;
-  // the bias towards sampling from more recent data.
-  // Per Cormode et al. an alpha of 0.015 strongly biases to the last 5 minutes
-  private static final double DEFAULT_ALPHA = 0.015;
+  // Double buffer the two FastLongHistograms.
+  // As they are reset they learn how the buckets should be spaced
+  // So keep two around and use them
+  protected final FastLongHistogram histogram;
 
   protected final String name;
   protected final String desc;
-  private final Reservoir reservoir;
-  private final AtomicLong min;
-  private final AtomicLong max;
-  private final AtomicLong sum;
-  private final AtomicLong count;
+  protected final Counter counter = new Counter(0);
 
   public MutableHistogram(MetricsInfo info) {
     this(info.name(), info.description());
   }
 
   public MutableHistogram(String name, String description) {
+    this(name, description, Integer.MAX_VALUE << 2);
+  }
+
+  protected MutableHistogram(String name, String description, long maxExpected) {
     this.name = StringUtils.capitalize(name);
     this.desc = StringUtils.uncapitalize(description);
-    reservoir = new ExponentiallyDecayingReservoir(DEFAULT_SAMPLE_SIZE, DEFAULT_ALPHA);
-    count = new AtomicLong();
-    min = new AtomicLong(Long.MAX_VALUE);
-    max = new AtomicLong(Long.MIN_VALUE);
-    sum = new AtomicLong();
+    this.histogram = new FastLongHistogram(FastLongHistogram.DEFAULT_NBINS, 1, maxExpected);
   }
 
   public void add(final long val) {
-    setChanged();
-    count.incrementAndGet();
-    reservoir.update(val);
-    setMax(val);
-    setMin(val);
-    sum.getAndAdd(val);
-  }
-
-  private void setMax(final long potentialMax) {
-    boolean done = false;
-    while (!done) {
-      final long currentMax = max.get();
-      done = currentMax >= potentialMax
-          || max.compareAndSet(currentMax, potentialMax);
-    }
-  }
-
-  private void setMin(long potentialMin) {
-    boolean done = false;
-    while (!done) {
-      final long currentMin = min.get();
-      done = currentMin <= potentialMin
-          || min.compareAndSet(currentMin, potentialMin);
-    }
-  }
-
-  public long getMax() {
-    if (count.get() > 0) {
-      return max.get();
-    }
-    return 0L;
-  }
-
-  public long getMin() {
-    if (count.get() > 0) {
-      return min.get();
-    }
-    return 0L;
-  }
-
-  public double getMean() {
-    long cCount = count.get();
-    if (cCount > 0) {
-      return sum.get() / (double) cCount;
-    }
-    return 0.0;
+    counter.increment();
+    histogram.add(val, 1);
   }
 
   @Override
-  public void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
-    if (all || changed()) {
-      clearChanged();
-      updateSnapshotMetrics(metricsRecordBuilder);
-    }
+  public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
+    // Get a reference to the old histogram.
+    FastLongHistogram histo = histogram.reset();
+    updateSnapshotMetrics(metricsRecordBuilder, histo);
   }
 
-  public void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder) {
-      final Snapshot s = reservoir.getSnapshot();
-      metricsRecordBuilder.addCounter(Interns.info(name + NUM_OPS_METRIC_NAME, desc), count.get());
-
-      metricsRecordBuilder.addGauge(Interns.info(name + MIN_METRIC_NAME, desc), getMin());
-      metricsRecordBuilder.addGauge(Interns.info(name + MAX_METRIC_NAME, desc), getMax());
-      metricsRecordBuilder.addGauge(Interns.info(name + MEAN_METRIC_NAME, desc), getMean());
-
-      metricsRecordBuilder.addGauge(Interns.info(name + MEDIAN_METRIC_NAME, desc), s.getMedian());
-      metricsRecordBuilder.addGauge(Interns.info(name + SEVENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
-          s.get75thPercentile());
-      metricsRecordBuilder.addGauge(Interns.info(name + NINETIETH_PERCENTILE_METRIC_NAME, desc),
-          s.getValue(0.90));
-      metricsRecordBuilder.addGauge(Interns.info(name + NINETY_FIFTH_PERCENTILE_METRIC_NAME, desc),
-          s.get95thPercentile());
-      metricsRecordBuilder.addGauge(Interns.info(name + NINETY_NINETH_PERCENTILE_METRIC_NAME, desc),
-          s.get99thPercentile());
+  protected void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder,
+                                       FastLongHistogram histo) {
+    metricsRecordBuilder.addCounter(Interns.info(name + NUM_OPS_METRIC_NAME, desc), counter.get());
+    metricsRecordBuilder.addGauge(Interns.info(name + MIN_METRIC_NAME, desc), histo.getMin());
+    metricsRecordBuilder.addGauge(Interns.info(name + MAX_METRIC_NAME, desc), histo.getMax());
+    metricsRecordBuilder.addGauge(Interns.info(name + MEAN_METRIC_NAME, desc), histo.getMean());
+
+    long[] percentiles = histo.getQuantiles();
+
+    metricsRecordBuilder.addGauge(Interns.info(name + TWENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+        percentiles[0]);
+    metricsRecordBuilder.addGauge(Interns.info(name + MEDIAN_METRIC_NAME, desc),
+        percentiles[1]);
+    metricsRecordBuilder.addGauge(Interns.info(name + SEVENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+        percentiles[2]);
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETIETH_PERCENTILE_METRIC_NAME, desc),
+        percentiles[3]);
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+        percentiles[4]);
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETY_EIGHTH_PERCENTILE_METRIC_NAME, desc),
+        percentiles[5]);
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETY_NINETH_PERCENTILE_METRIC_NAME, desc),
+        percentiles[6]);
+    metricsRecordBuilder.addGauge(
+        Interns.info(name + NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME, desc),
+        percentiles[7]);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
index ac1f497..ac8aee0 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
@@ -18,26 +18,29 @@
 
 package org.apache.hadoop.metrics2.lib;
 
-import java.util.concurrent.atomic.AtomicLongArray;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.FastLongHistogram;
+import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-
 /**
  * Extended histogram implementation with metric range counters.
  */
 @InterfaceAudience.Private
-public abstract class MutableRangeHistogram extends MutableHistogram {
+public abstract class MutableRangeHistogram extends MutableHistogram implements MetricHistogram {
 
   public MutableRangeHistogram(MetricsInfo info) {
     this(info.name(), info.description());
   }
 
   public MutableRangeHistogram(String name, String description) {
-    super(name, description);    
+    this(name, description, Integer.MAX_VALUE << 2);
   }
-  
+
+  public MutableRangeHistogram(String name, String description, long expectedMax) {
+    super(name, description, expectedMax);
+  }
+
   /**
    * Returns the type of range histogram size or time 
    */
@@ -46,49 +49,39 @@ public abstract class MutableRangeHistogram extends MutableHistogram {
   /**
    * Returns the ranges to be counted 
    */
-  public abstract long[] getRange();
-  
-  /**
-   * Returns the range counts 
-   */
-  public abstract AtomicLongArray getRangeVals();
+  public abstract long[] getRanges();
 
-  @Override
-  public void add(final long val) {
-    super.add(val);
-    updateBand(val);
-  }
-
-  private void updateBand(final long val) {
-    int i;
-    for (i=0; i<getRange().length && val > getRange()[i]; i++);
-    getRangeVals().incrementAndGet(i);
-  }
   
   @Override
-  public void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
-    if (all || changed()) {
-      clearChanged();
-      updateSnapshotMetrics(metricsRecordBuilder);
-      updateSnapshotRangeMetrics(metricsRecordBuilder);
-    }
+  public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
+    // Get a reference to the old histogram.
+    FastLongHistogram histo = histogram.reset();
+    updateSnapshotMetrics(metricsRecordBuilder, histo);
+    updateSnapshotRangeMetrics(metricsRecordBuilder, histo);
   }
-  
-  public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder) {
-    long prior = 0;
-    for (int i = 0; i < getRange().length; i++) {
-      long val = getRangeVals().get(i);
-      if (val > 0) {
+
+  public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder,
+                                         FastLongHistogram histogram) {
+    long priorRange = 0;
+    long cumNum = 0;
+
+    final long[] ranges = getRanges();
+    final String rangeType = getRangeType();
+    for (int i = 0; i < ranges.length - 1; i++) {
+      long val = histogram.getNumAtOrBelow(ranges[i]);
+      if (val - cumNum > 0) {
         metricsRecordBuilder.addCounter(
-          Interns.info(name + "_" + getRangeType() + "_" + prior + "-" + getRange()[i], desc), val);
+            Interns.info(name + "_" + rangeType + "_" + priorRange + "-" + ranges[i], desc),
+            val - cumNum);
       }
-      prior = getRange()[i];
+      priorRange = ranges[i];
+      cumNum = val;
     }
-    long val = getRangeVals().get(getRange().length);
-    if (val > 0) {
+    long val = histogram.getCount();
+    if (val - cumNum > 0) {
       metricsRecordBuilder.addCounter(
-        Interns.info(name + "_" + getRangeType() + "_" + getRange()[getRange().length - 1] + "-inf", desc),
-        getRangeVals().get(getRange().length));
+          Interns.info(name + "_" + rangeType + "_" + ranges[ranges.length - 1] + "-inf", desc),
+          val - cumNum);
     }
   }  
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java
index 2f1d57a..38e78a2 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.metrics2.lib;
 
-import java.util.concurrent.atomic.AtomicLongArray;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.metrics2.MetricsInfo;
 
@@ -28,30 +26,29 @@ import org.apache.hadoop.metrics2.MetricsInfo;
  */
 @InterfaceAudience.Private
 public class MutableSizeHistogram extends MutableRangeHistogram {
-  private final String rangeType = "SizeRangeCount";
-  private final long[] ranges = {10,100,1000,10000,100000,1000000,10000000,100000000};
-  private final AtomicLongArray rangeVals = new AtomicLongArray(getRange().length+1);
+  private final static String RANGE_TYPE = "SizeRangeCount";
+  private final static long[] RANGES = {10,100,1000,10000,100000,1000000,10000000,100000000};
 
   public MutableSizeHistogram(MetricsInfo info) {
     this(info.name(), info.description());
   }
 
   public MutableSizeHistogram(String name, String description) {
-    super(name, description);
+    this(name, description, RANGES[RANGES.length-2]);
+  }
+
+  public MutableSizeHistogram(String name, String description, long expectedMax) {
+    super(name, description, expectedMax);
   }
 
   @Override
   public String getRangeType() {
-    return rangeType;
+    return RANGE_TYPE;
   }
 
   @Override
-  public long[] getRange() {
-    return ranges;
+  public long[] getRanges() {
+    return RANGES;
   }
-  
-  @Override
-  public AtomicLongArray getRangeVals() {
-    return rangeVals;
-  }  
+
 }


[26/37] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by sy...@apache.org.
HBASE-15128 Disable region splits and merges switch in master


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

Branch: refs/heads/hbase-12439
Commit: 24d481c5803e69a6190339cd8bb218b2c4585459
Parents: 75c57a0
Author: chenheng <ch...@apache.org>
Authored: Fri Feb 26 08:11:16 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Fri Feb 26 08:11:16 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |   27 +-
 .../hbase/client/ConnectionImplementation.java  |   14 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   31 +
 .../hadoop/hbase/protobuf/RequestConverter.java |   49 +
 .../hbase/zookeeper/ZooKeeperWatcher.java       |   10 +
 .../hbase/protobuf/generated/MasterProtos.java  | 4304 ++++++++++++++----
 .../protobuf/generated/SnapshotProtos.java      |  500 +-
 .../protobuf/generated/ZooKeeperProtos.java     |  462 +-
 hbase-protocol/src/main/protobuf/Master.proto   |   36 +
 .../src/main/protobuf/ZooKeeper.proto           |    7 +
 .../hadoop/hbase/master/AssignmentManager.java  |   10 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   28 +
 .../hadoop/hbase/master/MasterRpcServices.java  |   42 +
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   35 +
 .../zookeeper/SplitOrMergeTrackerManager.java   |  151 +
 .../hbase/client/TestSplitOrMergeStatus.java    |  198 +
 hbase-shell/src/main/ruby/hbase/admin.rb        |   32 +
 hbase-shell/src/main/ruby/shell.rb              |    2 +
 .../ruby/shell/commands/splitormerge_enabled.rb |   41 +
 .../ruby/shell/commands/splitormerge_switch.rb  |   43 +
 20 files changed, 4822 insertions(+), 1200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index d7b52d5..c3b524b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -1678,11 +1678,28 @@ public interface Admin extends Abortable, Closeable {
   List<SecurityCapability> getSecurityCapabilities() throws IOException;
 
   /**
+   * Turn the Split or Merge switches on or off.
+   *
+   * @param enabled enabled or not
+   * @param synchronous If true, it waits until current split() call, if outstanding, to return.
+   * @param switchTypes switchType list {@link MasterSwitchType}
+   * @return Previous switch value array
+   */
+  boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
+                                   final MasterSwitchType... switchTypes) throws IOException;
+
+  /**
+   * Query the current state of the switch
+   *
+   * @return true if the switch is enabled, false otherwise.
+   */
+  boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException;
+
+  /**
    * Currently, there are only two compact types:
    * {@code NORMAL} means do store files compaction;
    * {@code MOB} means do mob files compaction.
    * */
-
   @InterfaceAudience.Public
   @InterfaceStability.Unstable
   public enum CompactType {
@@ -1692,4 +1709,12 @@ public interface Admin extends Abortable, Closeable {
 
     CompactType(int value) {}
   }
+  
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public enum MasterSwitchType {
+    SPLIT,
+    MERGE
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index dfa9937..64eb9fb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1742,6 +1742,20 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }
 
       @Override
+      public MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(
+        RpcController controller, MasterProtos.SetSplitOrMergeEnabledRequest request)
+        throws ServiceException {
+        return stub.setSplitOrMergeEnabled(controller, request);
+      }
+
+      @Override
+      public MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(
+        RpcController controller, MasterProtos.IsSplitOrMergeEnabledRequest request)
+              throws ServiceException {
+        return stub.isSplitOrMergeEnabled(controller, request);
+      }
+
+      @Override
       public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
           IsNormalizerEnabledRequest request) throws ServiceException {
         return stub.isNormalizerEnabled(controller, request);

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index db94ff4..c2a0bb8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescripti
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
@@ -3378,6 +3379,36 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  @Override
+  public boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
+                                          final MasterSwitchType... switchTypes)
+    throws IOException {
+    return executeCallable(new MasterCallable<boolean[]>(getConnection()) {
+      @Override
+      public boolean[] call(int callTimeout) throws ServiceException {
+        MasterProtos.SetSplitOrMergeEnabledResponse response = master.setSplitOrMergeEnabled(null,
+          RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchTypes));
+        boolean[] result = new boolean[switchTypes.length];
+        int i = 0;
+        for (Boolean prevValue : response.getPrevValueList()) {
+          result[i++] = prevValue;
+        }
+        return result;
+      }
+    });
+  }
+
+  @Override
+  public boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+      @Override
+      public Boolean call(int callTimeout) throws ServiceException {
+        return master.isSplitOrMergeEnabled(null,
+          RequestConverter.buildIsSplitOrMergeEnabledRequest(switchType)).getEnabled();
+      }
+    });
+  }
+
   private HRegionInfo getMobRegionInfo(TableName tableName) {
     return new HRegionInfo(tableName, Bytes.toBytes(".mob"),
             HConstants.EMPTY_END_ROW, false, 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 572d92c..99e993d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Action;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -76,6 +77,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
@@ -95,6 +97,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabled
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
@@ -103,6 +106,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequ
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
@@ -1692,4 +1696,49 @@ public final class RequestConverter {
   public static SetNormalizerRunningRequest buildSetNormalizerRunningRequest(boolean on) {
     return SetNormalizerRunningRequest.newBuilder().setOn(on).build();
   }
+
+  /**
+   * Creates a protocol buffer IsSplitOrMergeEnabledRequest
+   *
+   * @param switchType see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}
+   * @return a IsSplitOrMergeEnabledRequest
+   */
+  public static IsSplitOrMergeEnabledRequest buildIsSplitOrMergeEnabledRequest(
+    Admin.MasterSwitchType switchType) {
+    IsSplitOrMergeEnabledRequest.Builder builder = IsSplitOrMergeEnabledRequest.newBuilder();
+    builder.setSwitchType(convert(switchType));
+    return builder.build();
+  }
+
+  /**
+   * Creates a protocol buffer SetSplitOrMergeEnabledRequest
+   *
+   * @param enabled switch is enabled or not
+   * @param synchronous set switch sync?
+   * @param switchTypes see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}, it is
+   *                    a list.
+   * @return a SetSplitOrMergeEnabledRequest
+   */
+  public static SetSplitOrMergeEnabledRequest buildSetSplitOrMergeEnabledRequest(boolean enabled,
+    boolean synchronous, Admin.MasterSwitchType... switchTypes) {
+    SetSplitOrMergeEnabledRequest.Builder builder = SetSplitOrMergeEnabledRequest.newBuilder();
+    builder.setEnabled(enabled);
+    builder.setSynchronous(synchronous);
+    for (Admin.MasterSwitchType switchType : switchTypes) {
+      builder.addSwitchTypes(convert(switchType));
+    }
+    return builder.build();
+  }
+
+  private static MasterProtos.MasterSwitchType convert(Admin.MasterSwitchType switchType) {
+    switch (switchType) {
+      case SPLIT:
+        return MasterProtos.MasterSwitchType.SPLIT;
+      case MERGE:
+        return MasterProtos.MasterSwitchType.MERGE;
+      default:
+        break;
+    }
+    throw new UnsupportedOperationException("Unsupport switch type:" + switchType);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index 36a9bc5..b665353 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -115,6 +115,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
   public String balancerZNode;
   // znode containing the state of region normalizer
   private String regionNormalizerZNode;
+  // znode containing the state of all switches, currently there are split and merge child node.
+  private String switchZNode;
   // znode containing the lock for the tables
   public String tableLockZNode;
   // znode containing the state of recovering regions
@@ -430,6 +432,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
         conf.get("zookeeper.znode.balancer", "balancer"));
     regionNormalizerZNode = ZKUtil.joinZNode(baseZNode,
       conf.get("zookeeper.znode.regionNormalizer", "normalizer"));
+    switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch"));
     tableLockZNode = ZKUtil.joinZNode(baseZNode,
         conf.get("zookeeper.znode.tableLock", "table-lock"));
     recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
@@ -789,4 +792,11 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
   public String getRegionNormalizerZNode() {
     return regionNormalizerZNode;
   }
+
+  /**
+   *  @return ZK node for switch
+   * */
+  public String getSwitchZNode() {
+    return switchZNode;
+  }
 }


[29/37] hbase git commit: Revert "HBASE-15128 Disable region splits and merges switch in master"

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 073eba9..043d549 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -8,88 +8,6 @@ public final class MasterProtos {
   public static void registerAllExtensions(
       com.google.protobuf.ExtensionRegistry registry) {
   }
-  /**
-   * Protobuf enum {@code hbase.pb.MasterSwitchType}
-   */
-  public enum MasterSwitchType
-      implements com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>SPLIT = 0;</code>
-     */
-    SPLIT(0, 0),
-    /**
-     * <code>MERGE = 1;</code>
-     */
-    MERGE(1, 1),
-    ;
-
-    /**
-     * <code>SPLIT = 0;</code>
-     */
-    public static final int SPLIT_VALUE = 0;
-    /**
-     * <code>MERGE = 1;</code>
-     */
-    public static final int MERGE_VALUE = 1;
-
-
-    public final int getNumber() { return value; }
-
-    public static MasterSwitchType valueOf(int value) {
-      switch (value) {
-        case 0: return SPLIT;
-        case 1: return MERGE;
-        default: return null;
-      }
-    }
-
-    public static com.google.protobuf.Internal.EnumLiteMap<MasterSwitchType>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static com.google.protobuf.Internal.EnumLiteMap<MasterSwitchType>
-        internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap<MasterSwitchType>() {
-            public MasterSwitchType findValueByNumber(int number) {
-              return MasterSwitchType.valueOf(number);
-            }
-          };
-
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
-        getValueDescriptor() {
-      return getDescriptor().getValues().get(index);
-    }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.getDescriptor().getEnumTypes().get(0);
-    }
-
-    private static final MasterSwitchType[] VALUES = values();
-
-    public static MasterSwitchType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-      if (desc.getType() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "EnumValueDescriptor is not for this type.");
-      }
-      return VALUES[desc.getIndex()];
-    }
-
-    private final int index;
-    private final int value;
-
-    private MasterSwitchType(int index, int value) {
-      this.index = index;
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:hbase.pb.MasterSwitchType)
-  }
-
   public interface AddColumnRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -28846,62 +28764,28 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.IsBalancerEnabledResponse)
   }
 
-  public interface SetSplitOrMergeEnabledRequestOrBuilder
+  public interface NormalizeRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
-
-    // required bool enabled = 1;
-    /**
-     * <code>required bool enabled = 1;</code>
-     */
-    boolean hasEnabled();
-    /**
-     * <code>required bool enabled = 1;</code>
-     */
-    boolean getEnabled();
-
-    // optional bool synchronous = 2;
-    /**
-     * <code>optional bool synchronous = 2;</code>
-     */
-    boolean hasSynchronous();
-    /**
-     * <code>optional bool synchronous = 2;</code>
-     */
-    boolean getSynchronous();
-
-    // repeated .hbase.pb.MasterSwitchType switch_types = 3;
-    /**
-     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> getSwitchTypesList();
-    /**
-     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-     */
-    int getSwitchTypesCount();
-    /**
-     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchTypes(int index);
   }
   /**
-   * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledRequest}
+   * Protobuf type {@code hbase.pb.NormalizeRequest}
    */
-  public static final class SetSplitOrMergeEnabledRequest extends
+  public static final class NormalizeRequest extends
       com.google.protobuf.GeneratedMessage
-      implements SetSplitOrMergeEnabledRequestOrBuilder {
-    // Use SetSplitOrMergeEnabledRequest.newBuilder() to construct.
-    private SetSplitOrMergeEnabledRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements NormalizeRequestOrBuilder {
+    // Use NormalizeRequest.newBuilder() to construct.
+    private NormalizeRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private SetSplitOrMergeEnabledRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private NormalizeRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final SetSplitOrMergeEnabledRequest defaultInstance;
-    public static SetSplitOrMergeEnabledRequest getDefaultInstance() {
+    private static final NormalizeRequest defaultInstance;
+    public static NormalizeRequest getDefaultInstance() {
       return defaultInstance;
     }
 
-    public SetSplitOrMergeEnabledRequest getDefaultInstanceForType() {
+    public NormalizeRequest getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -28911,12 +28795,11 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private SetSplitOrMergeEnabledRequest(
+    private NormalizeRequest(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
-      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -28934,49 +28817,6 @@ public final class MasterProtos {
               }
               break;
             }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              enabled_ = input.readBool();
-              break;
-            }
-            case 16: {
-              bitField0_ |= 0x00000002;
-              synchronous_ = input.readBool();
-              break;
-            }
-            case 24: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(3, rawValue);
-              } else {
-                if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                  switchTypes_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType>();
-                  mutable_bitField0_ |= 0x00000004;
-                }
-                switchTypes_.add(value);
-              }
-              break;
-            }
-            case 26: {
-              int length = input.readRawVarint32();
-              int oldLimit = input.pushLimit(length);
-              while(input.getBytesUntilLimit() > 0) {
-                int rawValue = input.readEnum();
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.valueOf(rawValue);
-                if (value == null) {
-                  unknownFields.mergeVarintField(3, rawValue);
-                } else {
-                  if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                    switchTypes_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType>();
-                    mutable_bitField0_ |= 0x00000004;
-                  }
-                  switchTypes_.add(value);
-                }
-              }
-              input.popLimit(oldLimit);
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -28985,109 +28825,44 @@ public final class MasterProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-          switchTypes_ = java.util.Collections.unmodifiableList(switchTypes_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<SetSplitOrMergeEnabledRequest> PARSER =
-        new com.google.protobuf.AbstractParser<SetSplitOrMergeEnabledRequest>() {
-      public SetSplitOrMergeEnabledRequest parsePartialFrom(
+    public static com.google.protobuf.Parser<NormalizeRequest> PARSER =
+        new com.google.protobuf.AbstractParser<NormalizeRequest>() {
+      public NormalizeRequest parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new SetSplitOrMergeEnabledRequest(input, extensionRegistry);
+        return new NormalizeRequest(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<SetSplitOrMergeEnabledRequest> getParserForType() {
+    public com.google.protobuf.Parser<NormalizeRequest> getParserForType() {
       return PARSER;
     }
 
-    private int bitField0_;
-    // required bool enabled = 1;
-    public static final int ENABLED_FIELD_NUMBER = 1;
-    private boolean enabled_;
-    /**
-     * <code>required bool enabled = 1;</code>
-     */
-    public boolean hasEnabled() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required bool enabled = 1;</code>
-     */
-    public boolean getEnabled() {
-      return enabled_;
-    }
-
-    // optional bool synchronous = 2;
-    public static final int SYNCHRONOUS_FIELD_NUMBER = 2;
-    private boolean synchronous_;
-    /**
-     * <code>optional bool synchronous = 2;</code>
-     */
-    public boolean hasSynchronous() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional bool synchronous = 2;</code>
-     */
-    public boolean getSynchronous() {
-      return synchronous_;
-    }
-
-    // repeated .hbase.pb.MasterSwitchType switch_types = 3;
-    public static final int SWITCH_TYPES_FIELD_NUMBER = 3;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> switchTypes_;
-    /**
-     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> getSwitchTypesList() {
-      return switchTypes_;
-    }
-    /**
-     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-     */
-    public int getSwitchTypesCount() {
-      return switchTypes_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchTypes(int index) {
-      return switchTypes_.get(index);
-    }
-
     private void initFields() {
-      enabled_ = false;
-      synchronous_ = false;
-      switchTypes_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasEnabled()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -29095,15 +28870,6 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBool(1, enabled_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBool(2, synchronous_);
-      }
-      for (int i = 0; i < switchTypes_.size(); i++) {
-        output.writeEnum(3, switchTypes_.get(i).getNumber());
-      }
       getUnknownFields().writeTo(output);
     }
 
@@ -29113,23 +28879,6 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(1, enabled_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(2, synchronous_);
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < switchTypes_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
-            .computeEnumSizeNoTag(switchTypes_.get(i).getNumber());
-        }
-        size += dataSize;
-        size += 1 * switchTypes_.size();
-      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -29147,24 +28896,12 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest) obj;
 
       boolean result = true;
-      result = result && (hasEnabled() == other.hasEnabled());
-      if (hasEnabled()) {
-        result = result && (getEnabled()
-            == other.getEnabled());
-      }
-      result = result && (hasSynchronous() == other.hasSynchronous());
-      if (hasSynchronous()) {
-        result = result && (getSynchronous()
-            == other.getSynchronous());
-      }
-      result = result && getSwitchTypesList()
-          .equals(other.getSwitchTypesList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -29178,70 +28915,58 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasEnabled()) {
-        hash = (37 * hash) + ENABLED_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getEnabled());
-      }
-      if (hasSynchronous()) {
-        hash = (37 * hash) + SYNCHRONOUS_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getSynchronous());
-      }
-      if (getSwitchTypesCount() > 0) {
-        hash = (37 * hash) + SWITCH_TYPES_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnumList(getSwitchTypesList());
-      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -29250,7 +28975,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -29262,24 +28987,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledRequest}
+     * Protobuf type {@code hbase.pb.NormalizeRequest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequestOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -29299,12 +29024,6 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
-        enabled_ = false;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        synchronous_ = false;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        switchTypes_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -29314,79 +29033,43 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeRequest_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.enabled_ = enabled_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.synchronous_ = synchronous_;
-        if (((bitField0_ & 0x00000004) == 0x00000004)) {
-          switchTypes_ = java.util.Collections.unmodifiableList(switchTypes_);
-          bitField0_ = (bitField0_ & ~0x00000004);
-        }
-        result.switchTypes_ = switchTypes_;
-        result.bitField0_ = to_bitField0_;
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest(this);
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest.getDefaultInstance()) return this;
-        if (other.hasEnabled()) {
-          setEnabled(other.getEnabled());
-        }
-        if (other.hasSynchronous()) {
-          setSynchronous(other.getSynchronous());
-        }
-        if (!other.switchTypes_.isEmpty()) {
-          if (switchTypes_.isEmpty()) {
-            switchTypes_ = other.switchTypes_;
-            bitField0_ = (bitField0_ & ~0x00000004);
-          } else {
-            ensureSwitchTypesIsMutable();
-            switchTypes_.addAll(other.switchTypes_);
-          }
-          onChanged();
-        }
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest.getDefaultInstance()) return this;
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
-        if (!hasEnabled()) {
-          
-          return false;
-        }
         return true;
       }
 
@@ -29394,11 +29077,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -29407,193 +29090,50 @@ public final class MasterProtos {
         }
         return this;
       }
-      private int bitField0_;
-
-      // required bool enabled = 1;
-      private boolean enabled_ ;
-      /**
-       * <code>required bool enabled = 1;</code>
-       */
-      public boolean hasEnabled() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required bool enabled = 1;</code>
-       */
-      public boolean getEnabled() {
-        return enabled_;
-      }
-      /**
-       * <code>required bool enabled = 1;</code>
-       */
-      public Builder setEnabled(boolean value) {
-        bitField0_ |= 0x00000001;
-        enabled_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required bool enabled = 1;</code>
-       */
-      public Builder clearEnabled() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        enabled_ = false;
-        onChanged();
-        return this;
-      }
-
-      // optional bool synchronous = 2;
-      private boolean synchronous_ ;
-      /**
-       * <code>optional bool synchronous = 2;</code>
-       */
-      public boolean hasSynchronous() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional bool synchronous = 2;</code>
-       */
-      public boolean getSynchronous() {
-        return synchronous_;
-      }
-      /**
-       * <code>optional bool synchronous = 2;</code>
-       */
-      public Builder setSynchronous(boolean value) {
-        bitField0_ |= 0x00000002;
-        synchronous_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool synchronous = 2;</code>
-       */
-      public Builder clearSynchronous() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        synchronous_ = false;
-        onChanged();
-        return this;
-      }
-
-      // repeated .hbase.pb.MasterSwitchType switch_types = 3;
-      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> switchTypes_ =
-        java.util.Collections.emptyList();
-      private void ensureSwitchTypesIsMutable() {
-        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
-          switchTypes_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType>(switchTypes_);
-          bitField0_ |= 0x00000004;
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> getSwitchTypesList() {
-        return java.util.Collections.unmodifiableList(switchTypes_);
-      }
-      /**
-       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-       */
-      public int getSwitchTypesCount() {
-        return switchTypes_.size();
-      }
-      /**
-       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchTypes(int index) {
-        return switchTypes_.get(index);
-      }
-      /**
-       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-       */
-      public Builder setSwitchTypes(
-          int index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        ensureSwitchTypesIsMutable();
-        switchTypes_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-       */
-      public Builder addSwitchTypes(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        ensureSwitchTypesIsMutable();
-        switchTypes_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-       */
-      public Builder addAllSwitchTypes(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType> values) {
-        ensureSwitchTypesIsMutable();
-        super.addAll(values, switchTypes_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.MasterSwitchType switch_types = 3;</code>
-       */
-      public Builder clearSwitchTypes() {
-        switchTypes_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000004);
-        onChanged();
-        return this;
-      }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SetSplitOrMergeEnabledRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.NormalizeRequest)
     }
 
     static {
-      defaultInstance = new SetSplitOrMergeEnabledRequest(true);
+      defaultInstance = new NormalizeRequest(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.SetSplitOrMergeEnabledRequest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.NormalizeRequest)
   }
 
-  public interface SetSplitOrMergeEnabledResponseOrBuilder
+  public interface NormalizeResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // repeated bool prev_value = 1;
-    /**
-     * <code>repeated bool prev_value = 1;</code>
-     */
-    java.util.List<java.lang.Boolean> getPrevValueList();
+    // required bool normalizer_ran = 1;
     /**
-     * <code>repeated bool prev_value = 1;</code>
+     * <code>required bool normalizer_ran = 1;</code>
      */
-    int getPrevValueCount();
+    boolean hasNormalizerRan();
     /**
-     * <code>repeated bool prev_value = 1;</code>
+     * <code>required bool normalizer_ran = 1;</code>
      */
-    boolean getPrevValue(int index);
+    boolean getNormalizerRan();
   }
   /**
-   * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledResponse}
+   * Protobuf type {@code hbase.pb.NormalizeResponse}
    */
-  public static final class SetSplitOrMergeEnabledResponse extends
+  public static final class NormalizeResponse extends
       com.google.protobuf.GeneratedMessage
-      implements SetSplitOrMergeEnabledResponseOrBuilder {
-    // Use SetSplitOrMergeEnabledResponse.newBuilder() to construct.
-    private SetSplitOrMergeEnabledResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements NormalizeResponseOrBuilder {
+    // Use NormalizeResponse.newBuilder() to construct.
+    private NormalizeResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private SetSplitOrMergeEnabledResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private NormalizeResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final SetSplitOrMergeEnabledResponse defaultInstance;
-    public static SetSplitOrMergeEnabledResponse getDefaultInstance() {
+    private static final NormalizeResponse defaultInstance;
+    public static NormalizeResponse getDefaultInstance() {
       return defaultInstance;
     }
 
-    public SetSplitOrMergeEnabledResponse getDefaultInstanceForType() {
+    public NormalizeResponse getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -29603,7 +29143,7 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private SetSplitOrMergeEnabledResponse(
+    private NormalizeResponse(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -29627,24 +29167,8 @@ public final class MasterProtos {
               break;
             }
             case 8: {
-              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                prevValue_ = new java.util.ArrayList<java.lang.Boolean>();
-                mutable_bitField0_ |= 0x00000001;
-              }
-              prevValue_.add(input.readBool());
-              break;
-            }
-            case 10: {
-              int length = input.readRawVarint32();
-              int limit = input.pushLimit(length);
-              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001) && input.getBytesUntilLimit() > 0) {
-                prevValue_ = new java.util.ArrayList<java.lang.Boolean>();
-                mutable_bitField0_ |= 0x00000001;
-              }
-              while (input.getBytesUntilLimit() > 0) {
-                prevValue_.add(input.readBool());
-              }
-              input.popLimit(limit);
+              bitField0_ |= 0x00000001;
+              normalizerRan_ = input.readBool();
               break;
             }
           }
@@ -29655,71 +29179,66 @@ public final class MasterProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-          prevValue_ = java.util.Collections.unmodifiableList(prevValue_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<SetSplitOrMergeEnabledResponse> PARSER =
-        new com.google.protobuf.AbstractParser<SetSplitOrMergeEnabledResponse>() {
-      public SetSplitOrMergeEnabledResponse parsePartialFrom(
+    public static com.google.protobuf.Parser<NormalizeResponse> PARSER =
+        new com.google.protobuf.AbstractParser<NormalizeResponse>() {
+      public NormalizeResponse parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new SetSplitOrMergeEnabledResponse(input, extensionRegistry);
+        return new NormalizeResponse(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<SetSplitOrMergeEnabledResponse> getParserForType() {
+    public com.google.protobuf.Parser<NormalizeResponse> getParserForType() {
       return PARSER;
     }
 
-    // repeated bool prev_value = 1;
-    public static final int PREV_VALUE_FIELD_NUMBER = 1;
-    private java.util.List<java.lang.Boolean> prevValue_;
-    /**
-     * <code>repeated bool prev_value = 1;</code>
-     */
-    public java.util.List<java.lang.Boolean>
-        getPrevValueList() {
-      return prevValue_;
-    }
+    private int bitField0_;
+    // required bool normalizer_ran = 1;
+    public static final int NORMALIZER_RAN_FIELD_NUMBER = 1;
+    private boolean normalizerRan_;
     /**
-     * <code>repeated bool prev_value = 1;</code>
+     * <code>required bool normalizer_ran = 1;</code>
      */
-    public int getPrevValueCount() {
-      return prevValue_.size();
+    public boolean hasNormalizerRan() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>repeated bool prev_value = 1;</code>
+     * <code>required bool normalizer_ran = 1;</code>
      */
-    public boolean getPrevValue(int index) {
-      return prevValue_.get(index);
+    public boolean getNormalizerRan() {
+      return normalizerRan_;
     }
 
     private void initFields() {
-      prevValue_ = java.util.Collections.emptyList();
+      normalizerRan_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
+      if (!hasNormalizerRan()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -29727,8 +29246,8 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
-      for (int i = 0; i < prevValue_.size(); i++) {
-        output.writeBool(1, prevValue_.get(i));
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBool(1, normalizerRan_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -29739,11 +29258,9 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
-      {
-        int dataSize = 0;
-        dataSize = 1 * getPrevValueList().size();
-        size += dataSize;
-        size += 1 * getPrevValueList().size();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(1, normalizerRan_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -29762,14 +29279,17 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse) obj;
 
       boolean result = true;
-      result = result && getPrevValueList()
-          .equals(other.getPrevValueList());
+      result = result && (hasNormalizerRan() == other.hasNormalizerRan());
+      if (hasNormalizerRan()) {
+        result = result && (getNormalizerRan()
+            == other.getNormalizerRan());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -29783,62 +29303,62 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (getPrevValueCount() > 0) {
-        hash = (37 * hash) + PREV_VALUE_FIELD_NUMBER;
-        hash = (53 * hash) + getPrevValueList().hashCode();
+      if (hasNormalizerRan()) {
+        hash = (37 * hash) + NORMALIZER_RAN_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getNormalizerRan());
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -29847,7 +29367,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -29859,24 +29379,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.SetSplitOrMergeEnabledResponse}
+     * Protobuf type {@code hbase.pb.NormalizeResponse}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponseOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponseOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -29896,7 +29416,7 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
-        prevValue_ = java.util.Collections.emptyList();
+        normalizerRan_ = false;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
@@ -29907,59 +29427,57 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetSplitOrMergeEnabledResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_NormalizeResponse_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse(this);
         int from_bitField0_ = bitField0_;
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          prevValue_ = java.util.Collections.unmodifiableList(prevValue_);
-          bitField0_ = (bitField0_ & ~0x00000001);
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
         }
-        result.prevValue_ = prevValue_;
+        result.normalizerRan_ = normalizerRan_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse.getDefaultInstance()) return this;
-        if (!other.prevValue_.isEmpty()) {
-          if (prevValue_.isEmpty()) {
-            prevValue_ = other.prevValue_;
-            bitField0_ = (bitField0_ & ~0x00000001);
-          } else {
-            ensurePrevValueIsMutable();
-            prevValue_.addAll(other.prevValue_);
-          }
-          onChanged();
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse.getDefaultInstance()) return this;
+        if (other.hasNormalizerRan()) {
+          setNormalizerRan(other.getNormalizerRan());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
+        if (!hasNormalizerRan()) {
+          
+          return false;
+        }
         return true;
       }
 
@@ -29967,11 +29485,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -29982,115 +29500,82 @@ public final class MasterProtos {
       }
       private int bitField0_;
 
-      // repeated bool prev_value = 1;
-      private java.util.List<java.lang.Boolean> prevValue_ = java.util.Collections.emptyList();
-      private void ensurePrevValueIsMutable() {
-        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          prevValue_ = new java.util.ArrayList<java.lang.Boolean>(prevValue_);
-          bitField0_ |= 0x00000001;
-         }
-      }
-      /**
-       * <code>repeated bool prev_value = 1;</code>
-       */
-      public java.util.List<java.lang.Boolean>
-          getPrevValueList() {
-        return java.util.Collections.unmodifiableList(prevValue_);
-      }
-      /**
-       * <code>repeated bool prev_value = 1;</code>
-       */
-      public int getPrevValueCount() {
-        return prevValue_.size();
-      }
-      /**
-       * <code>repeated bool prev_value = 1;</code>
-       */
-      public boolean getPrevValue(int index) {
-        return prevValue_.get(index);
-      }
+      // required bool normalizer_ran = 1;
+      private boolean normalizerRan_ ;
       /**
-       * <code>repeated bool prev_value = 1;</code>
+       * <code>required bool normalizer_ran = 1;</code>
        */
-      public Builder setPrevValue(
-          int index, boolean value) {
-        ensurePrevValueIsMutable();
-        prevValue_.set(index, value);
-        onChanged();
-        return this;
+      public boolean hasNormalizerRan() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>repeated bool prev_value = 1;</code>
+       * <code>required bool normalizer_ran = 1;</code>
        */
-      public Builder addPrevValue(boolean value) {
-        ensurePrevValueIsMutable();
-        prevValue_.add(value);
-        onChanged();
-        return this;
+      public boolean getNormalizerRan() {
+        return normalizerRan_;
       }
       /**
-       * <code>repeated bool prev_value = 1;</code>
+       * <code>required bool normalizer_ran = 1;</code>
        */
-      public Builder addAllPrevValue(
-          java.lang.Iterable<? extends java.lang.Boolean> values) {
-        ensurePrevValueIsMutable();
-        super.addAll(values, prevValue_);
+      public Builder setNormalizerRan(boolean value) {
+        bitField0_ |= 0x00000001;
+        normalizerRan_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>repeated bool prev_value = 1;</code>
+       * <code>required bool normalizer_ran = 1;</code>
        */
-      public Builder clearPrevValue() {
-        prevValue_ = java.util.Collections.emptyList();
+      public Builder clearNormalizerRan() {
         bitField0_ = (bitField0_ & ~0x00000001);
+        normalizerRan_ = false;
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SetSplitOrMergeEnabledResponse)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.NormalizeResponse)
     }
 
     static {
-      defaultInstance = new SetSplitOrMergeEnabledResponse(true);
+      defaultInstance = new NormalizeResponse(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.SetSplitOrMergeEnabledResponse)
+    // @@protoc_insertion_point(class_scope:hbase.pb.NormalizeResponse)
   }
 
-  public interface IsSplitOrMergeEnabledRequestOrBuilder
+  public interface SetNormalizerRunningRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .hbase.pb.MasterSwitchType switch_type = 1;
+    // required bool on = 1;
     /**
-     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+     * <code>required bool on = 1;</code>
      */
-    boolean hasSwitchType();
+    boolean hasOn();
     /**
-     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+     * <code>required bool on = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchType();
+    boolean getOn();
   }
   /**
-   * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledRequest}
+   * Protobuf type {@code hbase.pb.SetNormalizerRunningRequest}
    */
-  public static final class IsSplitOrMergeEnabledRequest extends
+  public static final class SetNormalizerRunningRequest extends
       com.google.protobuf.GeneratedMessage
-      implements IsSplitOrMergeEnabledRequestOrBuilder {
-    // Use IsSplitOrMergeEnabledRequest.newBuilder() to construct.
-    private IsSplitOrMergeEnabledRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements SetNormalizerRunningRequestOrBuilder {
+    // Use SetNormalizerRunningRequest.newBuilder() to construct.
+    private SetNormalizerRunningRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private IsSplitOrMergeEnabledRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private SetNormalizerRunningRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final IsSplitOrMergeEnabledRequest defaultInstance;
-    public static IsSplitOrMergeEnabledRequest getDefaultInstance() {
+    private static final SetNormalizerRunningRequest defaultInstance;
+    public static SetNormalizerRunningRequest getDefaultInstance() {
       return defaultInstance;
     }
 
-    public IsSplitOrMergeEnabledRequest getDefaultInstanceForType() {
+    public SetNormalizerRunningRequest getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -30100,7 +29585,7 @@ public final class MasterProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private IsSplitOrMergeEnabledRequest(
+    private SetNormalizerRunningRequest(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -30124,14 +29609,8 @@ public final class MasterProtos {
               break;
             }
             case 8: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(1, rawValue);
-              } else {
-                bitField0_ |= 0x00000001;
-                switchType_ = value;
-              }
+              bitField0_ |= 0x00000001;
+              on_ = input.readBool();
               break;
             }
           }
@@ -30148,57 +29627,57 @@ public final class MasterProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<IsSplitOrMergeEnabledRequest> PARSER =
-        new com.google.protobuf.AbstractParser<IsSplitOrMergeEnabledRequest>() {
-      public IsSplitOrMergeEnabledRequest parsePartialFrom(
+    public static com.google.protobuf.Parser<SetNormalizerRunningRequest> PARSER =
+        new com.google.protobuf.AbstractParser<SetNormalizerRunningRequest>() {
+      public SetNormalizerRunningRequest parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new IsSplitOrMergeEnabledRequest(input, extensionRegistry);
+        return new SetNormalizerRunningRequest(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<IsSplitOrMergeEnabledRequest> getParserForType() {
+    public com.google.protobuf.Parser<SetNormalizerRunningRequest> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // required .hbase.pb.MasterSwitchType switch_type = 1;
-    public static final int SWITCH_TYPE_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType switchType_;
+    // required bool on = 1;
+    public static final int ON_FIELD_NUMBER = 1;
+    private boolean on_;
     /**
-     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+     * <code>required bool on = 1;</code>
      */
-    public boolean hasSwitchType() {
+    public boolean hasOn() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+     * <code>required bool on = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchType() {
-      return switchType_;
+    public boolean getOn() {
+      return on_;
     }
 
     private void initFields() {
-      switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
+      on_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasSwitchType()) {
+      if (!hasOn()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -30210,7 +29689,7 @@ public final class MasterProtos {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeEnum(1, switchType_.getNumber());
+        output.writeBool(1, on_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -30223,7 +29702,7 @@ public final class MasterProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(1, switchType_.getNumber());
+          .computeBoolSize(1, on_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -30242,16 +29721,16 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) obj;
 
       boolean result = true;
-      result = result && (hasSwitchType() == other.hasSwitchType());
-      if (hasSwitchType()) {
-        result = result &&
-            (getSwitchType() == other.getSwitchType());
+      result = result && (hasOn() == other.hasOn());
+      if (hasOn()) {
+        result = result && (getOn()
+            == other.getOn());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -30266,62 +29745,62 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasSwitchType()) {
-        hash = (37 * hash) + SWITCH_TYPE_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnum(getSwitchType());
+      if (hasOn()) {
+        hash = (37 * hash) + ON_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getOn());
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -30330,7 +29809,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -30342,24 +29821,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledRequest}
+     * Protobuf type {@code hbase.pb.SetNormalizerRunningRequest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequestOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -30379,7 +29858,7 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
-        switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
+        on_ = false;
         bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
@@ -30390,54 +29869,54 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_IsSplitOrMergeEnabledRequest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SetNormalizerRunningRequest_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.switchType_ = switchType_;
+        result.on_ = on_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest.getDefaultInstance()) return this;
-        if (other.hasSwitchType()) {
-          setSwitchType(other.getSwitchType());
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest.getDefaultInstance()) return this;
+        if (other.hasOn()) {
+          setOn(other.getOn());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
-        if (!hasSwitchType()) {
+        if (!hasOn()) {
           
           return false;
         }
@@ -30448,11 +29927,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -30463,85 +29942,82 @@ public final class MasterProtos {
       }
       private int bitField0_;
 
-      // required .hbase.pb.MasterSwitchType switch_type = 1;
-      private org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
+      // required bool on = 1;
+      private boolean on_ ;
       /**
-       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+       * <code>required bool on = 1;</code>
        */
-      public boolean hasSwitchType() {
+      public boolean hasOn() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+       * <code>required bool on = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType getSwitchType() {
-        return switchType_;
+      public boolean getOn() {
+        return on_;
       }
       /**
-       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+       * <code>required bool on = 1;</code>
        */
-      public Builder setSwitchType(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
+      public Builder setOn(boolean value) {
         bitField0_ |= 0x00000001;
-        switchType_ = value;
+        on_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required .hbase.pb.MasterSwitchType switch_type = 1;</code>
+       * <code>required bool on = 1;</code>
        */
-      public Builder clearSwitchType() {
+      public Builder clearOn() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        switchType_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterSwitchType.SPLIT;
+        on_ = false;
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.IsSplitOrMergeEnabledRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SetNormalizerRunningRequest)
     }
 
     static {
-      defaultInstance = new IsSplitOrMergeEnabledRequest(true);
+      defaultInstance = new SetNormalizerRunningRequest(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.IsSplitOrMergeEnabledRequest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.SetNormalizerRunningRequest)
   }
 
-  public interface IsSplitOrMergeEnabledResponseOrBuilder
+  public interface SetNormalizerRunningResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required bool enabled = 1;
+    // optional bool prev_normalizer_value = 1;
     /**
-     * <code>required bool enabled = 1;</code>
+     * <code>optional bool prev_normalizer_value = 1;</code>
      */
-    boolean hasEnabled();
+    boolean hasPrevNormalizerValue();
     /**
-     * <code>required bool enabled = 1;</code>
+     * <code>optional bool prev_normalizer_value = 1;</code>
      */
-    boolean getEnabled();
+    boolean getPrevNormalizerValue();
   }
   /**
-   * Protobuf type {@code hbase.pb.IsSplitOrMergeEnabledResponse}
+   * Protobuf type {@code hbase.pb.SetNormalizerRunningResponse}
    */
-  public static final class IsSplitOrMergeEnabledResponse extends
+  public static final class SetNormalizerRunningResponse extends
       com.google.protobuf.GeneratedMessage
-      implements IsSplitOrMergeEnabledResponseOrBuilder {
-    // Use IsSplitOrMergeEnabledResponse.newBuilder() to construct.
-    private IsSplitOrMergeEnabledResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements SetNormalizerRunningResponseOrBuilder {
+    // Use SetNormalizerRunningResponse.newBuilder() to construct.
+    private SetNormalizerRunningResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private IsSplitOrMergeEnabledResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private SetNormalizerRunningResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final IsSplitOrMergeEnabledResponse defaultInstance;
-    public static IsSplitOrMergeEnabledResponse getDefaultInstance() {
+    private static final SetNormalizerRunningResponse defaultInstance;
+    public static SetNormalizerRunningResponse getDefaultInstance() {
       return defaultInstance;
     }
 
-    public IsSplitOrMergeEnabledResponse getDefaultInstanceForType() {
+    public SetNormalizerRunningRes

<TRUNCATED>

[06/37] hbase git commit: HBASE-15306 Make RPC call queue length dynamically configurable

Posted by sy...@apache.org.
HBASE-15306 Make RPC call queue length dynamically configurable


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

Branch: refs/heads/hbase-12439
Commit: f47dba74d498d5d39f124ad8ea5723c437acbc85
Parents: 58283fa
Author: Mikhail Antonov <an...@apache.org>
Authored: Tue Feb 23 14:20:40 2016 -0800
Committer: Mikhail Antonov <an...@apache.org>
Committed: Tue Feb 23 14:20:40 2016 -0800

----------------------------------------------------------------------
 .../hbase/ipc/BalancedQueueRpcExecutor.java     | 11 +++++-
 .../hadoop/hbase/ipc/RWQueueRpcExecutor.java    | 19 +++++++++-
 .../apache/hadoop/hbase/ipc/RpcExecutor.java    | 11 ++++++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  3 ++
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    | 18 ++++++++-
 .../hbase/ipc/TestSimpleRpcScheduler.java       | 39 ++++++++++++++++++++
 6 files changed, 97 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f47dba74/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
index 79b4ec8..e4205eb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
@@ -66,6 +66,10 @@ public class BalancedQueueRpcExecutor extends RpcExecutor {
 
   protected void initializeQueues(final int numQueues,
       final Class<? extends BlockingQueue> queueClass, Object... initargs) {
+    if (initargs.length > 0) {
+      currentQueueLimit = (int) initargs[0];
+      initargs[0] = Math.max((int) initargs[0], DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT);
+    }
     for (int i = 0; i < numQueues; ++i) {
       queues.add((BlockingQueue<CallRunner>) ReflectionUtils.newInstance(queueClass, initargs));
     }
@@ -74,7 +78,12 @@ public class BalancedQueueRpcExecutor extends RpcExecutor {
   @Override
   public boolean dispatch(final CallRunner callTask) throws InterruptedException {
     int queueIndex = balancer.getNextQueue();
-    return queues.get(queueIndex).offer(callTask);
+    BlockingQueue<CallRunner> queue = queues.get(queueIndex);
+    // that means we can overflow by at most <num reader> size (5), that's ok
+    if (queue.size() >= currentQueueLimit) {
+      return false;
+    }
+    return queue.offer(callTask);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f47dba74/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
index 544370d..a9648b0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java
@@ -139,12 +139,22 @@ public class RWQueueRpcExecutor extends RpcExecutor {
               " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount +
               ((numScanQueues == 0) ? "" : " scanQueues=" + numScanQueues +
                 " scanHandlers=" + scanHandlersCount));
-
+    if (writeQueueInitArgs.length > 0) {
+      currentQueueLimit = (int) writeQueueInitArgs[0];
+      writeQueueInitArgs[0] = Math.max((int) writeQueueInitArgs[0],
+        DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT);
+    }
     for (int i = 0; i < numWriteQueues; ++i) {
+
       queues.add((BlockingQueue<CallRunner>)
         ReflectionUtils.newInstance(writeQueueClass, writeQueueInitArgs));
     }
 
+    if (readQueueInitArgs.length > 0) {
+      currentQueueLimit = (int) readQueueInitArgs[0];
+      readQueueInitArgs[0] = Math.max((int) readQueueInitArgs[0],
+        DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT);
+    }
     for (int i = 0; i < (numReadQueues + numScanQueues); ++i) {
       queues.add((BlockingQueue<CallRunner>)
         ReflectionUtils.newInstance(readQueueClass, readQueueInitArgs));
@@ -170,7 +180,12 @@ public class RWQueueRpcExecutor extends RpcExecutor {
     } else {
       queueIndex = numWriteQueues + readBalancer.getNextQueue();
     }
-    return queues.get(queueIndex).offer(callTask);
+
+    BlockingQueue<CallRunner> queue = queues.get(queueIndex);
+    if (queue.size() >= currentQueueLimit) {
+      return false;
+    }
+    return queue.offer(callTask);
   }
 
   private boolean isWriteRequest(final RequestHeader header, final Message param) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f47dba74/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
index 017bf39..22cb195 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
@@ -42,6 +42,9 @@ import com.google.common.base.Strings;
 public abstract class RpcExecutor {
   private static final Log LOG = LogFactory.getLog(RpcExecutor.class);
 
+  protected static final int DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT = 250;
+  protected volatile int currentQueueLimit;
+
   private final AtomicInteger activeHandlerCount = new AtomicInteger(0);
   private final List<Thread> handlers;
   private final int handlerCount;
@@ -210,4 +213,12 @@ public abstract class RpcExecutor {
       return ThreadLocalRandom.current().nextInt(queueSize);
     }
   }
+
+  /**
+   * Update current soft limit for executor's call queues
+   * @param conf updated configuration
+   */
+  public void resizeQueues(Configuration conf) {
+    currentQueueLimit = conf.getInt("hbase.ipc.server.max.callqueue.length", currentQueueLimit);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f47dba74/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 58fc598..6ddfb9a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -2099,6 +2099,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   @Override
   public void onConfigurationChange(Configuration newConf) {
     initReconfigurable(newConf);
+    if (scheduler instanceof ConfigurationObserver) {
+      ((ConfigurationObserver)scheduler).onConfigurationChange(newConf);
+    }
   }
 
   private void initReconfigurable(Configuration confToLoad) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f47dba74/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index 8de714d..0003254 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
 
 /**
@@ -36,7 +37,7 @@ import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 @InterfaceStability.Evolving
-public class SimpleRpcScheduler extends RpcScheduler {
+public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObserver {
   private static final Log LOG = LogFactory.getLog(SimpleRpcScheduler.class);
 
   public static final String CALL_QUEUE_READ_SHARE_CONF_KEY =
@@ -56,6 +57,21 @@ public class SimpleRpcScheduler extends RpcScheduler {
       = "hbase.ipc.server.queue.max.call.delay";
 
   /**
+   * Resize call queues;
+   * @param conf new configuration
+   */
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    callExecutor.resizeQueues(conf);
+    if (priorityExecutor != null) {
+      priorityExecutor.resizeQueues(conf);
+    }
+    if (replicationExecutor != null) {
+      replicationExecutor.resizeQueues(conf);
+    }
+  }
+
+  /**
    * Comparator used by the "normal callQueue" if DEADLINE_CALL_QUEUE_CONF_KEY is set to true.
    * It uses the calculated "deadline" e.g. to deprioritize long-running job
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/f47dba74/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index db992cd..66032e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -56,7 +56,9 @@ import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.eq;
@@ -323,4 +325,41 @@ public class TestSimpleRpcScheduler {
       }
     }).when(callTask).run();
   }
+
+  @Test
+  public void testSoftAndHardQueueLimits() throws Exception {
+    Configuration schedConf = HBaseConfiguration.create();
+
+    schedConf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 0);
+    schedConf.setInt("hbase.ipc.server.max.callqueue.length", 5);
+
+    PriorityFunction priority = mock(PriorityFunction.class);
+    when(priority.getPriority(any(RequestHeader.class), any(Message.class),
+      any(User.class))).thenReturn(HConstants.NORMAL_QOS);
+    SimpleRpcScheduler scheduler = new SimpleRpcScheduler(schedConf, 0, 0, 0, priority,
+      HConstants.QOS_THRESHOLD);
+    try {
+      scheduler.start();
+
+      CallRunner putCallTask = mock(CallRunner.class);
+      RpcServer.Call putCall = mock(RpcServer.Call.class);
+      putCall.param = RequestConverter.buildMutateRequest(
+        Bytes.toBytes("abc"), new Put(Bytes.toBytes("row")));
+      RequestHeader putHead = RequestHeader.newBuilder().setMethodName("mutate").build();
+      when(putCallTask.getCall()).thenReturn(putCall);
+      when(putCall.getHeader()).thenReturn(putHead);
+
+      assertTrue(scheduler.dispatch(putCallTask));
+
+      schedConf.setInt("hbase.ipc.server.max.callqueue.length", 0);
+      scheduler.onConfigurationChange(schedConf);
+      assertFalse(scheduler.dispatch(putCallTask));
+
+      schedConf.setInt("hbase.ipc.server.max.callqueue.length", 1);
+      scheduler.onConfigurationChange(schedConf);
+      assertTrue(scheduler.dispatch(putCallTask));
+    } finally {
+      scheduler.stop();
+    }
+  }
 }


[12/37] hbase git commit: HBASE-15302 Reenable the other tests disabled by HBASE-14678

Posted by sy...@apache.org.
HBASE-15302 Reenable the other tests disabled by HBASE-14678

Signed-off-by: stack <st...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 30cec72f9ade972d7e9ce4bba527b0e6074cae60
Parents: 876a6ab
Author: Phil Yang <ud...@gmail.com>
Authored: Mon Feb 22 14:17:24 2016 +0800
Committer: stack <st...@apache.org>
Committed: Wed Feb 24 07:14:01 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/wal/WALSplitter.java    |   11 +-
 .../hbase/TestPartialResultsFromClientSide.java |  832 ++++++++
 .../TestMobSnapshotCloneIndependence.java       |   69 +
 .../client/TestSnapshotCloneIndependence.java   |  481 +++++
 .../master/TestDistributedLogSplitting.java     | 1799 ++++++++++++++++++
 .../balancer/TestStochasticLoadBalancer2.java   |   90 +
 .../TestMasterFailoverWithProcedures.java       |  514 +++++
 .../TestMobFlushSnapshotFromClient.java         |   72 +
 .../apache/hadoop/hbase/wal/TestWALSplit.java   | 1320 +++++++++++++
 .../hbase/wal/TestWALSplitCompressed.java       |   36 +
 .../hbase/client/TestReplicationShell.java      |   37 +
 11 files changed, 5256 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 54b82b2..010fd37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -515,13 +515,14 @@ public class WALSplitter {
    * @param fs
    * @param logEntry
    * @param rootDir HBase root dir.
-   * @param fileBeingSplit the file being split currently. Used to generate tmp file name.
+   * @param fileNameBeingSplit the file being split currently. Used to generate tmp file name.
    * @return Path to file into which to dump split log edits.
    * @throws IOException
    */
   @SuppressWarnings("deprecation")
-  private static Path getRegionSplitEditsPath(final FileSystem fs,
-      final Entry logEntry, final Path rootDir, FileStatus fileBeingSplit)
+  @VisibleForTesting
+  static Path getRegionSplitEditsPath(final FileSystem fs,
+      final Entry logEntry, final Path rootDir, String fileNameBeingSplit)
   throws IOException {
     Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
     String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
@@ -556,7 +557,7 @@ public class WALSplitter {
     // Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
     // region's replayRecoveredEdits will not delete it
     String fileName = formatRecoveredEditsFileName(logEntry.getKey().getSequenceId());
-    fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileBeingSplit.getPath().getName());
+    fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileNameBeingSplit);
     return new Path(dir, fileName);
   }
 
@@ -1518,7 +1519,7 @@ public class WALSplitter {
      * @return a path with a write for that path. caller should close.
      */
     private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir) throws IOException {
-      Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, fileBeingSplit);
+      Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, fileBeingSplit.getPath().getName());
       if (regionedits == null) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
new file mode 100644
index 0000000..a6f8373
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -0,0 +1,832 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.ClientScanner;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
+import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.FirstKeyValueMatchingQualifiersFilter;
+import org.apache.hadoop.hbase.filter.RandomRowFilter;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * These tests are focused on testing how partial results appear to a client. Partial results are
+ * {@link Result}s that contain only a portion of a row's complete list of cells. Partial results
+ * are formed when the server breaches its maximum result size when trying to service a client's RPC
+ * request. It is the responsibility of the scanner on the client side to recognize when partial
+ * results have been returned and to take action to form the complete results.
+ * <p>
+ * Unless the flag {@link Scan#setAllowPartialResults(boolean)} has been set to true, the caller of
+ * {@link ResultScanner#next()} should never see partial results.
+ */
+@Category(MediumTests.class)
+public class TestPartialResultsFromClientSide {
+  private static final Log LOG = LogFactory.getLog(TestPartialResultsFromClientSide.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static Table TABLE = null;
+
+  /**
+   * Table configuration
+   */
+  private static TableName TABLE_NAME = TableName.valueOf("testTable");
+
+  private static int NUM_ROWS = 5;
+  private static byte[] ROW = Bytes.toBytes("testRow");
+  private static byte[][] ROWS = HTestConst.makeNAscii(ROW, NUM_ROWS);
+
+  // Should keep this value below 10 to keep generation of expected kv's simple. If above 10 then
+  // table/row/cf1/... will be followed by table/row/cf10/... instead of table/row/cf2/... which
+  // breaks the simple generation of expected kv's
+  private static int NUM_FAMILIES = 10;
+  private static byte[] FAMILY = Bytes.toBytes("testFamily");
+  private static byte[][] FAMILIES = HTestConst.makeNAscii(FAMILY, NUM_FAMILIES);
+
+  private static int NUM_QUALIFIERS = 10;
+  private static byte[] QUALIFIER = Bytes.toBytes("testQualifier");
+  private static byte[][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, NUM_QUALIFIERS);
+
+  private static int VALUE_SIZE = 1024;
+  private static byte[] VALUE = Bytes.createMaxByteArray(VALUE_SIZE);
+
+  private static int NUM_COLS = NUM_FAMILIES * NUM_QUALIFIERS;
+
+  // Approximation of how large the heap size of cells in our table. Should be accessed through
+  // getCellHeapSize().
+  private static long CELL_HEAP_SIZE = -1;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster(3);
+    TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
+  }
+
+  static Table createTestTable(TableName name, byte[][] rows, byte[][] families,
+      byte[][] qualifiers, byte[] cellValue) throws IOException {
+    Table ht = TEST_UTIL.createTable(name, families);
+    List<Put> puts = createPuts(rows, families, qualifiers, cellValue);
+    ht.put(puts);
+
+    return ht;
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Ensure that the expected key values appear in a result returned from a scanner that is
+   * combining partial results into complete results
+   * @throws Exception
+   */
+  @Test
+  public void testExpectedValuesOfPartialResults() throws Exception {
+    testExpectedValuesOfPartialResults(false);
+    testExpectedValuesOfPartialResults(true);
+  }
+
+  public void testExpectedValuesOfPartialResults(boolean reversed) throws Exception {
+    Scan partialScan = new Scan();
+    partialScan.setMaxVersions();
+    // Max result size of 1 ensures that each RPC request will return a single cell. The scanner
+    // will need to reconstruct the results into a complete result before returning to the caller
+    partialScan.setMaxResultSize(1);
+    partialScan.setReversed(reversed);
+    ResultScanner partialScanner = TABLE.getScanner(partialScan);
+
+    final int startRow = reversed ? ROWS.length - 1 : 0;
+    final int endRow = reversed ? -1 : ROWS.length;
+    final int loopDelta = reversed ? -1 : 1;
+    String message;
+
+    for (int row = startRow; row != endRow; row = row + loopDelta) {
+      message = "Ensuring the expected keyValues are present for row " + row;
+      List<Cell> expectedKeyValues = createKeyValuesForRow(ROWS[row], FAMILIES, QUALIFIERS, VALUE);
+      Result result = partialScanner.next();
+      assertFalse(result.isPartial());
+      verifyResult(result, expectedKeyValues, message);
+    }
+
+    partialScanner.close();
+  }
+
+  /**
+   * Ensure that we only see Results marked as partial when the allowPartial flag is set
+   * @throws Exception
+   */
+  @Test
+  public void testAllowPartialResults() throws Exception {
+    Scan scan = new Scan();
+    scan.setAllowPartialResults(true);
+    scan.setMaxResultSize(1);
+    ResultScanner scanner = TABLE.getScanner(scan);
+    Result result = scanner.next();
+
+    assertTrue(result != null);
+    assertTrue(result.isPartial());
+    assertTrue(result.rawCells() != null);
+    assertTrue(result.rawCells().length == 1);
+
+    scanner.close();
+
+    scan.setAllowPartialResults(false);
+    scanner = TABLE.getScanner(scan);
+    result = scanner.next();
+
+    assertTrue(result != null);
+    assertTrue(!result.isPartial());
+    assertTrue(result.rawCells() != null);
+    assertTrue(result.rawCells().length == NUM_COLS);
+
+    scanner.close();
+  }
+
+  /**
+   * Ensure that the results returned from a scanner that retrieves all results in a single RPC call
+   * matches the results that are returned from a scanner that must incrementally combine partial
+   * results into complete results. A variety of scan configurations can be tested
+   * @throws Exception
+   */
+  @Test
+  public void testEquivalenceOfScanResults() throws Exception {
+    Scan oneShotScan = new Scan();
+    oneShotScan.setMaxResultSize(Long.MAX_VALUE);
+
+    Scan partialScan = new Scan(oneShotScan);
+    partialScan.setMaxResultSize(1);
+
+    testEquivalenceOfScanResults(TABLE, oneShotScan, partialScan);
+  }
+
+  public void testEquivalenceOfScanResults(Table table, Scan scan1, Scan scan2) throws Exception {
+    ResultScanner scanner1 = table.getScanner(scan1);
+    ResultScanner scanner2 = table.getScanner(scan2);
+
+    Result r1 = null;
+    Result r2 = null;
+    int count = 0;
+
+    while ((r1 = scanner1.next()) != null) {
+      r2 = scanner2.next();
+
+      assertTrue(r2 != null);
+      compareResults(r1, r2, "Comparing result #" + count);
+      count++;
+    }
+
+    r2 = scanner2.next();
+    assertTrue("r2: " + r2 + " Should be null", r2 == null);
+
+    scanner1.close();
+    scanner2.close();
+  }
+
+  /**
+   * Order of cells in partial results matches the ordering of cells from complete results
+   * @throws Exception
+   */
+  @Test
+  public void testOrderingOfCellsInPartialResults() throws Exception {
+    Scan scan = new Scan();
+
+    for (int col = 1; col <= NUM_COLS; col++) {
+      scan.setMaxResultSize(getResultSizeForNumberOfCells(col));
+      testOrderingOfCellsInPartialResults(scan);
+
+      // Test again with a reversed scanner
+      scan.setReversed(true);
+      testOrderingOfCellsInPartialResults(scan);
+    }
+  }
+
+  public void testOrderingOfCellsInPartialResults(final Scan basePartialScan) throws Exception {
+    // Scan that retrieves results in pieces (partials). By setting allowPartialResults to be true
+    // the results will NOT be reconstructed and instead the caller will see the partial results
+    // returned by the server
+    Scan partialScan = new Scan(basePartialScan);
+    partialScan.setAllowPartialResults(true);
+    ResultScanner partialScanner = TABLE.getScanner(partialScan);
+
+    // Scan that retrieves all table results in single RPC request
+    Scan oneShotScan = new Scan(basePartialScan);
+    oneShotScan.setMaxResultSize(Long.MAX_VALUE);
+    oneShotScan.setCaching(ROWS.length);
+    ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan);
+
+    Result oneShotResult = oneShotScanner.next();
+    Result partialResult = null;
+    int iterationCount = 0;
+
+    while (oneShotResult != null && oneShotResult.rawCells() != null) {
+      List<Cell> aggregatePartialCells = new ArrayList<Cell>();
+      do {
+        partialResult = partialScanner.next();
+        assertTrue("Partial Result is null. iteration: " + iterationCount, partialResult != null);
+        assertTrue("Partial cells are null. iteration: " + iterationCount,
+            partialResult.rawCells() != null);
+
+        for (Cell c : partialResult.rawCells()) {
+          aggregatePartialCells.add(c);
+        }
+      } while (partialResult.isPartial());
+
+      assertTrue("Number of cells differs. iteration: " + iterationCount,
+          oneShotResult.rawCells().length == aggregatePartialCells.size());
+      final Cell[] oneShotCells = oneShotResult.rawCells();
+      for (int cell = 0; cell < oneShotCells.length; cell++) {
+        Cell oneShotCell = oneShotCells[cell];
+        Cell partialCell = aggregatePartialCells.get(cell);
+
+        assertTrue("One shot cell was null", oneShotCell != null);
+        assertTrue("Partial cell was null", partialCell != null);
+        assertTrue("Cell differs. oneShotCell:" + oneShotCell + " partialCell:" + partialCell,
+            oneShotCell.equals(partialCell));
+      }
+
+      oneShotResult = oneShotScanner.next();
+      iterationCount++;
+    }
+
+    assertTrue(partialScanner.next() == null);
+
+    partialScanner.close();
+    oneShotScanner.close();
+  }
+
+  /**
+   * Setting the max result size allows us to control how many cells we expect to see on each call
+   * to next on the scanner. Test a variety of different sizes for correctness
+   * @throws Exception
+   */
+  @Test
+  public void testExpectedNumberOfCellsPerPartialResult() throws Exception {
+    Scan scan = new Scan();
+    testExpectedNumberOfCellsPerPartialResult(scan);
+
+    scan.setReversed(true);
+    testExpectedNumberOfCellsPerPartialResult(scan);
+  }
+
+  public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan) throws Exception {
+    for (int expectedCells = 1; expectedCells <= NUM_COLS; expectedCells++) {
+      testExpectedNumberOfCellsPerPartialResult(baseScan, expectedCells);
+    }
+  }
+
+  public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan, int expectedNumberOfCells)
+      throws Exception {
+
+    if (LOG.isInfoEnabled()) LOG.info("groupSize:" + expectedNumberOfCells);
+
+    // Use the cellHeapSize to set maxResultSize such that we know how many cells to expect back
+    // from the call. The returned results should NOT exceed expectedNumberOfCells but may be less
+    // than it in cases where expectedNumberOfCells is not an exact multiple of the number of
+    // columns in the table.
+    Scan scan = new Scan(baseScan);
+    scan.setAllowPartialResults(true);
+    scan.setMaxResultSize(getResultSizeForNumberOfCells(expectedNumberOfCells));
+
+    ResultScanner scanner = TABLE.getScanner(scan);
+    Result result = null;
+    byte[] prevRow = null;
+    while ((result = scanner.next()) != null) {
+      assertTrue(result.rawCells() != null);
+
+      // Cases when cell count won't equal expectedNumberOfCells:
+      // 1. Returned result is the final result needed to form the complete result for that row
+      // 2. It is the first result we have seen for that row and thus may have been fetched as
+      // the last group of cells that fit inside the maxResultSize
+      assertTrue(
+          "Result's cell count differed from expected number. result: " + result,
+          result.rawCells().length == expectedNumberOfCells || !result.isPartial()
+              || !Bytes.equals(prevRow, result.getRow()));
+      prevRow = result.getRow();
+    }
+
+    scanner.close();
+  }
+
+  /**
+   * @return The approximate heap size of a cell in the test table. All cells should have
+   *         approximately the same heap size, so the value is cached to avoid repeating the
+   *         calculation
+   * @throws Exception
+   */
+  private long getCellHeapSize() throws Exception {
+    if (CELL_HEAP_SIZE == -1) {
+      // Do a partial scan that will return a single result with a single cell
+      Scan scan = new Scan();
+      scan.setMaxResultSize(1);
+      scan.setAllowPartialResults(true);
+      ResultScanner scanner = TABLE.getScanner(scan);
+
+      Result result = scanner.next();
+
+      assertTrue(result != null);
+      assertTrue(result.rawCells() != null);
+      assertTrue(result.rawCells().length == 1);
+
+      CELL_HEAP_SIZE = CellUtil.estimatedHeapSizeOf(result.rawCells()[0]);
+      if (LOG.isInfoEnabled()) LOG.info("Cell heap size: " + CELL_HEAP_SIZE);
+      scanner.close();
+    }
+
+    return CELL_HEAP_SIZE;
+  }
+
+  /**
+   * @param numberOfCells
+   * @return the result size that should be used in {@link Scan#setMaxResultSize(long)} if you want
+   *         the server to return exactly numberOfCells cells
+   * @throws Exception
+   */
+  private long getResultSizeForNumberOfCells(int numberOfCells) throws Exception {
+    return getCellHeapSize() * numberOfCells;
+  }
+
+  /**
+   * Test various combinations of batching and partial results for correctness
+   */
+  @Test
+  public void testPartialResultsAndBatch() throws Exception {
+    for (int batch = 1; batch <= NUM_COLS / 4; batch++) {
+      for (int cellsPerPartial = 1; cellsPerPartial <= NUM_COLS / 4; cellsPerPartial++) {
+        testPartialResultsAndBatch(batch, cellsPerPartial);
+      }
+    }
+  }
+
+  public void testPartialResultsAndBatch(final int batch, final int cellsPerPartialResult)
+      throws Exception {
+    if (LOG.isInfoEnabled()) {
+      LOG.info("batch: " + batch + " cellsPerPartialResult: " + cellsPerPartialResult);
+    }
+
+    Scan scan = new Scan();
+    scan.setMaxResultSize(getResultSizeForNumberOfCells(cellsPerPartialResult));
+    scan.setBatch(batch);
+    ResultScanner scanner = TABLE.getScanner(scan);
+    Result result = scanner.next();
+    int repCount = 0;
+
+    while ((result = scanner.next()) != null) {
+      assertTrue(result.rawCells() != null);
+
+      if (result.isPartial()) {
+        final String error =
+            "Cells:" + result.rawCells().length + " Batch size:" + batch
+                + " cellsPerPartialResult:" + cellsPerPartialResult + " rep:" + repCount;
+        assertTrue(error, result.rawCells().length <= Math.min(batch, cellsPerPartialResult));
+      } else {
+        assertTrue(result.rawCells().length <= batch);
+      }
+      repCount++;
+    }
+
+    scanner.close();
+  }
+
+  /**
+   * Test the method {@link Result#createCompleteResult(List, Result)}
+   * @throws Exception
+   */
+  @Test
+  public void testPartialResultsReassembly() throws Exception {
+    Scan scan = new Scan();
+    testPartialResultsReassembly(scan);
+    scan.setReversed(true);
+    testPartialResultsReassembly(scan);
+  }
+
+  public void testPartialResultsReassembly(Scan scanBase) throws Exception {
+    Scan partialScan = new Scan(scanBase);
+    partialScan.setMaxResultSize(1);
+    partialScan.setAllowPartialResults(true);
+    ResultScanner partialScanner = TABLE.getScanner(partialScan);
+
+    Scan oneShotScan = new Scan(scanBase);
+    oneShotScan.setMaxResultSize(Long.MAX_VALUE);
+    ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan);
+
+    ArrayList<Result> partials = new ArrayList<>();
+    for (int i = 0; i < NUM_ROWS; i++) {
+      Result partialResult = null;
+      Result completeResult = null;
+      Result oneShotResult = null;
+      partials.clear();
+
+      do {
+        partialResult = partialScanner.next();
+        partials.add(partialResult);
+      } while (partialResult != null && partialResult.isPartial());
+
+      completeResult = Result.createCompleteResult(partials);
+      oneShotResult = oneShotScanner.next();
+
+      compareResults(completeResult, oneShotResult, null);
+    }
+
+    assertTrue(oneShotScanner.next() == null);
+    assertTrue(partialScanner.next() == null);
+
+    oneShotScanner.close();
+    partialScanner.close();
+  }
+
+  /**
+   * When reconstructing the complete result from its partials we ensure that the row of each
+   * partial result is the same. If one of the rows differs, an exception is thrown.
+   */
+  @Test
+  public void testExceptionThrownOnMismatchedPartialResults() throws IOException {
+    assertTrue(NUM_ROWS >= 2);
+
+    ArrayList<Result> partials = new ArrayList<>();
+    Scan scan = new Scan();
+    scan.setMaxResultSize(Long.MAX_VALUE);
+    ResultScanner scanner = TABLE.getScanner(scan);
+    Result r1 = scanner.next();
+    partials.add(r1);
+    Result r2 = scanner.next();
+    partials.add(r2);
+
+    assertFalse(Bytes.equals(r1.getRow(), r2.getRow()));
+
+    try {
+      Result.createCompleteResult(partials);
+      fail("r1 and r2 are from different rows. It should not be possible to combine them into"
+          + " a single result");
+    } catch (IOException e) {
+    }
+
+    scanner.close();
+  }
+
+  /**
+   * When a scan has a filter where {@link org.apache.hadoop.hbase.filter.Filter#hasFilterRow()} is
+   * true, the scanner should not return partial results. The scanner cannot return partial results
+   * because the entire row needs to be read for the include/exclude decision to be made
+   */
+  @Test
+  public void testNoPartialResultsWhenRowFilterPresent() throws Exception {
+    Scan scan = new Scan();
+    scan.setMaxResultSize(1);
+    scan.setAllowPartialResults(true);
+    // If a filter hasFilter() is true then partial results should not be returned else filter
+    // application server side would break.
+    scan.setFilter(new RandomRowFilter(1.0f));
+    ResultScanner scanner = TABLE.getScanner(scan);
+
+    Result r = null;
+    while ((r = scanner.next()) != null) {
+      assertFalse(r.isPartial());
+    }
+
+    scanner.close();
+  }
+
+  /**
+   * Examine the interaction between the maxResultSize and caching. If the caching limit is reached
+   * before the maxResultSize limit, we should not see partial results. On the other hand, if the
+   * maxResultSize limit is reached before the caching limit, it is likely that partial results will
+   * be seen.
+   * @throws Exception
+   */
+  @Test
+  public void testPartialResultsAndCaching() throws Exception {
+    for (int caching = 1; caching <= NUM_ROWS; caching++) {
+      for (int maxResultRows = 0; maxResultRows <= NUM_ROWS; maxResultRows++) {
+        testPartialResultsAndCaching(maxResultRows, caching);
+      }
+    }
+  }
+
+  /**
+   * @param resultSizeRowLimit The row limit that will be enforced through maxResultSize
+   * @param cachingRowLimit The row limit that will be enforced through caching
+   * @throws Exception
+   */
+  public void testPartialResultsAndCaching(int resultSizeRowLimit, int cachingRowLimit)
+      throws Exception {
+    Scan scan = new Scan();
+    scan.setAllowPartialResults(true);
+
+    // The number of cells specified in the call to getResultSizeForNumberOfCells is offset to
+    // ensure that the result size we specify is not an exact multiple of the number of cells
+    // in a row. This ensures that partial results will be returned when the result size limit
+    // is reached before the caching limit.
+    int cellOffset = NUM_COLS / 3;
+    long maxResultSize = getResultSizeForNumberOfCells(resultSizeRowLimit * NUM_COLS + cellOffset);
+    scan.setMaxResultSize(maxResultSize);
+    scan.setCaching(cachingRowLimit);
+
+    ResultScanner scanner = TABLE.getScanner(scan);
+    ClientScanner clientScanner = (ClientScanner) scanner;
+    Result r = null;
+
+    // Approximate the number of rows we expect will fit into the specified max rsult size. If this
+    // approximation is less than caching, then we expect that the max result size limit will be
+    // hit before the caching limit and thus partial results may be seen
+    boolean expectToSeePartialResults = resultSizeRowLimit < cachingRowLimit;
+    while ((r = clientScanner.next()) != null) {
+      assertTrue(!r.isPartial() || expectToSeePartialResults);
+    }
+
+    scanner.close();
+  }
+
+  /**
+   * Small scans should not return partial results because it would prevent small scans from
+   * retrieving all of the necessary results in a single RPC request which is what makese small
+   * scans useful. Thus, ensure that even when {@link Scan#getAllowPartialResults()} is true, small
+   * scans do not return partial results
+   * @throws Exception
+   */
+  @Test
+  public void testSmallScansDoNotAllowPartials() throws Exception {
+    Scan scan = new Scan();
+    testSmallScansDoNotAllowPartials(scan);
+    scan.setReversed(true);
+    testSmallScansDoNotAllowPartials(scan);
+  }
+
+  public void testSmallScansDoNotAllowPartials(Scan baseScan) throws Exception {
+    Scan scan = new Scan(baseScan);
+    scan.setAllowPartialResults(true);
+    scan.setSmall(true);
+    scan.setMaxResultSize(1);
+
+    ResultScanner scanner = TABLE.getScanner(scan);
+    Result r = null;
+
+    while ((r = scanner.next()) != null) {
+      assertFalse(r.isPartial());
+    }
+
+    scanner.close();
+  }
+
+  /**
+   * Make puts to put the input value into each combination of row, family, and qualifier
+   * @param rows
+   * @param families
+   * @param qualifiers
+   * @param value
+   * @return
+   * @throws IOException
+   */
+  static ArrayList<Put> createPuts(byte[][] rows, byte[][] families, byte[][] qualifiers,
+      byte[] value) throws IOException {
+    Put put;
+    ArrayList<Put> puts = new ArrayList<>();
+
+    for (int row = 0; row < rows.length; row++) {
+      put = new Put(rows[row]);
+      for (int fam = 0; fam < families.length; fam++) {
+        for (int qual = 0; qual < qualifiers.length; qual++) {
+          KeyValue kv = new KeyValue(rows[row], families[fam], qualifiers[qual], qual, value);
+          put.add(kv);
+        }
+      }
+      puts.add(put);
+    }
+
+    return puts;
+  }
+
+  /**
+   * Make key values to represent each possible combination of family and qualifier in the specified
+   * row.
+   * @param row
+   * @param families
+   * @param qualifiers
+   * @param value
+   * @return
+   */
+  static ArrayList<Cell> createKeyValuesForRow(byte[] row, byte[][] families, byte[][] qualifiers,
+      byte[] value) {
+    ArrayList<Cell> outList = new ArrayList<>();
+    for (int fam = 0; fam < families.length; fam++) {
+      for (int qual = 0; qual < qualifiers.length; qual++) {
+        outList.add(new KeyValue(row, families[fam], qualifiers[qual], qual, value));
+      }
+    }
+    return outList;
+  }
+
+  /**
+   * Verifies that result contains all the key values within expKvList. Fails the test otherwise
+   * @param result
+   * @param expKvList
+   * @param msg
+   */
+  static void verifyResult(Result result, List<Cell> expKvList, String msg) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(msg);
+      LOG.info("Expected count: " + expKvList.size());
+      LOG.info("Actual count: " + result.size());
+    }
+
+    if (expKvList.size() == 0) return;
+
+    int i = 0;
+    for (Cell kv : result.rawCells()) {
+      if (i >= expKvList.size()) {
+        break; // we will check the size later
+      }
+
+      Cell kvExp = expKvList.get(i++);
+      assertTrue("Not equal. get kv: " + kv.toString() + " exp kv: " + kvExp.toString(),
+          kvExp.equals(kv));
+    }
+
+    assertEquals(expKvList.size(), result.size());
+  }
+
+  /**
+   * Compares two results and fails the test if the results are different
+   * @param r1
+   * @param r2
+   * @param message
+   */
+  static void compareResults(Result r1, Result r2, final String message) {
+    if (LOG.isInfoEnabled()) {
+      if (message != null) LOG.info(message);
+      LOG.info("r1: " + r1);
+      LOG.info("r2: " + r2);
+    }
+
+    final String failureMessage = "Results r1:" + r1 + " \nr2:" + r2 + " are not equivalent";
+    if (r1 == null && r2 == null) fail(failureMessage);
+    else if (r1 == null || r2 == null) fail(failureMessage);
+
+    try {
+      Result.compareResults(r1, r2);
+    } catch (Exception e) {
+      fail(failureMessage);
+    }
+  }
+
+  @Test
+  public void testReadPointAndPartialResults() throws Exception {
+    TableName testName = TableName.valueOf("testReadPointAndPartialResults");
+    int numRows = 5;
+    int numFamilies = 5;
+    int numQualifiers = 5;
+    byte[][] rows = HTestConst.makeNAscii(Bytes.toBytes("testRow"), numRows);
+    byte[][] families = HTestConst.makeNAscii(Bytes.toBytes("testFamily"), numFamilies);
+    byte[][] qualifiers = HTestConst.makeNAscii(Bytes.toBytes("testQualifier"), numQualifiers);
+    byte[] value = Bytes.createMaxByteArray(100);
+
+    Table tmpTable = createTestTable(testName, rows, families, qualifiers, value);
+
+    Scan scan = new Scan();
+    scan.setMaxResultSize(1);
+    scan.setAllowPartialResults(true);
+
+    // Open scanner before deletes
+    ResultScanner scanner = tmpTable.getScanner(scan);
+
+    Delete delete1 = new Delete(rows[0]);
+    delete1.addColumn(families[0], qualifiers[0], 0);
+    tmpTable.delete(delete1);
+
+    Delete delete2 = new Delete(rows[1]);
+    delete2.addColumn(families[1], qualifiers[1], 1);
+    tmpTable.delete(delete2);
+
+    // Should see all cells because scanner was opened prior to deletes
+    int scannerCount = countCellsFromScanner(scanner);
+    int expectedCount = numRows * numFamilies * numQualifiers;
+    assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+        scannerCount == expectedCount);
+
+    // Minus 2 for the two cells that were deleted
+    scanner = tmpTable.getScanner(scan);
+    scannerCount = countCellsFromScanner(scanner);
+    expectedCount = numRows * numFamilies * numQualifiers - 2;
+    assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+        scannerCount == expectedCount);
+
+    scanner = tmpTable.getScanner(scan);
+    // Put in 2 new rows. The timestamps differ from the deleted rows
+    Put put1 = new Put(rows[0]);
+    put1.add(new KeyValue(rows[0], families[0], qualifiers[0], 1, value));
+    tmpTable.put(put1);
+
+    Put put2 = new Put(rows[1]);
+    put2.add(new KeyValue(rows[1], families[1], qualifiers[1], 2, value));
+    tmpTable.put(put2);
+
+    // Scanner opened prior to puts. Cell count shouldn't have changed
+    scannerCount = countCellsFromScanner(scanner);
+    expectedCount = numRows * numFamilies * numQualifiers - 2;
+    assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+        scannerCount == expectedCount);
+
+    // Now the scanner should see the cells that were added by puts
+    scanner = tmpTable.getScanner(scan);
+    scannerCount = countCellsFromScanner(scanner);
+    expectedCount = numRows * numFamilies * numQualifiers;
+    assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+        scannerCount == expectedCount);
+
+    TEST_UTIL.deleteTable(testName);
+  }
+
+  /**
+   * Exhausts the scanner by calling next repetitively. Once completely exhausted, close scanner and
+   * return total cell count
+   * @param scanner
+   * @return
+   * @throws Exception
+   */
+  private int countCellsFromScanner(ResultScanner scanner) throws Exception {
+    Result result = null;
+    int numCells = 0;
+    while ((result = scanner.next()) != null) {
+      numCells += result.rawCells().length;
+    }
+
+    scanner.close();
+    return numCells;
+  }
+
+  /**
+   * Test partial Result re-assembly in the presence of different filters. The Results from the
+   * partial scanner should match the Results returned from a scanner that receives all of the
+   * results in one RPC to the server. The partial scanner is tested with a variety of different
+   * result sizes (all of which are less than the size necessary to fetch an entire row)
+   * @throws Exception
+   */
+  @Test
+  public void testPartialResultsWithColumnFilter() throws Exception {
+    testPartialResultsWithColumnFilter(new FirstKeyOnlyFilter());
+    testPartialResultsWithColumnFilter(new ColumnPrefixFilter(Bytes.toBytes("testQualifier5")));
+    testPartialResultsWithColumnFilter(new ColumnRangeFilter(Bytes.toBytes("testQualifer1"), true,
+        Bytes.toBytes("testQualifier7"), true));
+
+    Set<byte[]> qualifiers = new LinkedHashSet<>();
+    qualifiers.add(Bytes.toBytes("testQualifier5"));
+    testPartialResultsWithColumnFilter(new FirstKeyValueMatchingQualifiersFilter(qualifiers));
+  }
+
+  public void testPartialResultsWithColumnFilter(Filter filter) throws Exception {
+    assertTrue(!filter.hasFilterRow());
+
+    Scan partialScan = new Scan();
+    partialScan.setFilter(filter);
+
+    Scan oneshotScan = new Scan();
+    oneshotScan.setFilter(filter);
+    oneshotScan.setMaxResultSize(Long.MAX_VALUE);
+
+    for (int i = 1; i <= NUM_COLS; i++) {
+      partialScan.setMaxResultSize(getResultSizeForNumberOfCells(i));
+      testEquivalenceOfScanResults(TABLE, partialScan, oneshotScan);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
new file mode 100644
index 0000000..dcf20e5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
@@ -0,0 +1,69 @@
+/**
+ * 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.client;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test to verify that the cloned table is independent of the table from which it was cloned
+ */
+@Category(LargeTests.class)
+public class TestMobSnapshotCloneIndependence extends TestSnapshotCloneIndependence {
+  private static final Log LOG = LogFactory.getLog(TestMobSnapshotCloneIndependence.class);
+
+  /**
+   * Setup the config for the cluster and start it
+   * @throws Exception on failure
+   */
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(NUM_RS);
+  }
+
+  protected static void setupConf(Configuration conf) {
+    TestSnapshotCloneIndependence.setupConf(conf);
+    conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+  }
+
+  @Override
+  protected Table createTable(final TableName table, byte[] family) throws Exception {
+    return MobSnapshotTestingUtils.createMobTable(UTIL, table, family);
+  }
+
+  @Override
+  public void loadData(final Table table, byte[]... families) throws Exception {
+    SnapshotTestingUtils.loadData(UTIL, table.getName(), 1000, families);
+  }
+
+  @Override
+  protected int countRows(final Table table, final byte[]... families) throws Exception {
+    return MobSnapshotTestingUtils.countMobRows(table, families);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
new file mode 100644
index 0000000..0a3093b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -0,0 +1,481 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test to verify that the cloned table is independent of the table from which it was cloned
+ */
+@Category({LargeTests.class, ClientTests.class})
+public class TestSnapshotCloneIndependence {
+  private static final Log LOG = LogFactory.getLog(TestSnapshotCloneIndependence.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  protected static final int NUM_RS = 2;
+  private static final String STRING_TABLE_NAME = "test";
+  private static final String TEST_FAM_STR = "fam";
+  protected static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR);
+  protected static final TableName TABLE_NAME = TableName.valueOf(STRING_TABLE_NAME);
+  private static final int CLEANER_INTERVAL = 100;
+
+  /**
+   * Setup the config for the cluster and start it
+   * @throws Exception on fOailure
+   */
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(NUM_RS);
+  }
+
+  static void setupConf(Configuration conf) {
+    // Up the handlers; this test needs more than usual.
+    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 15);
+    // enable snapshot support
+    conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
+    // disable the ui
+    conf.setInt("hbase.regionsever.info.port", -1);
+    conf.setInt("hbase.master.info.port", -1);
+    // change the flush size to a small amount, regulating number of store files
+    conf.setInt("hbase.hregion.memstore.flush.size", 25000);
+    // so make sure we get a compaction when doing a load, but keep around
+    // some files in the store
+    conf.setInt("hbase.hstore.compaction.min", 10);
+    conf.setInt("hbase.hstore.compactionThreshold", 10);
+    // block writes if we get to 12 store files
+    conf.setInt("hbase.hstore.blockingStoreFiles", 12);
+    conf.setInt("hbase.regionserver.msginterval", 100);
+    conf.setBoolean("hbase.master.enabletable.roundrobin", true);
+    // Avoid potentially aggressive splitting which would cause snapshot to fail
+    conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
+        ConstantSizeRegionSplitPolicy.class.getName());
+    // Execute cleaner frequently to induce failures
+    conf.setInt("hbase.master.cleaner.interval", CLEANER_INTERVAL);
+    conf.setInt("hbase.master.hfilecleaner.plugins.snapshot.period", CLEANER_INTERVAL);
+    // Effectively disable TimeToLiveHFileCleaner. Don't want to fully disable it because that
+    // will even trigger races between creating the directory containing back references and
+    // the back reference itself.
+    conf.setInt("hbase.master.hfilecleaner.ttl", CLEANER_INTERVAL);
+  }
+
+  @Before
+  public void setup() throws Exception {
+    createTable(TABLE_NAME, TEST_FAM);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    UTIL.deleteTable(TABLE_NAME);
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  /**
+   * Verify that adding data to the cloned table will not affect the original, and vice-versa when
+   * it is taken as an online snapshot.
+   */
+  @Ignore ("Flakey. Fix") @Test (timeout=300000)
+  public void testOnlineSnapshotAppendIndependent() throws Exception {
+    runTestSnapshotAppendIndependent(true);
+  }
+
+  /**
+   * Verify that adding data to the cloned table will not affect the original, and vice-versa when
+   * it is taken as an offline snapshot.
+   */
+  @Test (timeout=300000)
+  @Ignore
+  public void testOfflineSnapshotAppendIndependent() throws Exception {
+    runTestSnapshotAppendIndependent(false);
+  }
+
+  /**
+   * Verify that adding metadata to the cloned table will not affect the original, and vice-versa
+   * when it is taken as an online snapshot.
+   */
+  @Test (timeout=300000)
+  public void testOnlineSnapshotMetadataChangesIndependent() throws Exception {
+    runTestSnapshotMetadataChangesIndependent(true);
+  }
+
+  /**
+   * Verify that adding netadata to the cloned table will not affect the original, and vice-versa
+   * when is taken as an online snapshot.
+   */
+  @Test (timeout=300000)
+  @Ignore
+  public void testOfflineSnapshotMetadataChangesIndependent() throws Exception {
+    runTestSnapshotMetadataChangesIndependent(false);
+  }
+
+  /**
+   * Verify that region operations, in this case splitting a region, are independent between the
+   * cloned table and the original.
+   */
+  @Test (timeout=300000)
+  @Ignore
+  public void testOfflineSnapshotRegionOperationsIndependent() throws Exception {
+    runTestRegionOperationsIndependent(false);
+  }
+
+  /**
+   * Verify that region operations, in this case splitting a region, are independent between the
+   * cloned table and the original.
+   */
+  @Test (timeout=300000)
+  public void testOnlineSnapshotRegionOperationsIndependent() throws Exception {
+    runTestRegionOperationsIndependent(true);
+  }
+
+  @Test (timeout=300000)
+  @Ignore
+  public void testOfflineSnapshotDeleteIndependent() throws Exception {
+    runTestSnapshotDeleteIndependent(false);
+  }
+
+  @Ignore ("Flakey test") @Test (timeout=300000)
+  public void testOnlineSnapshotDeleteIndependent() throws Exception {
+    runTestSnapshotDeleteIndependent(true);
+  }
+
+  private static void waitOnSplit(Connection c, final Table t, int originalCount) throws Exception {
+    for (int i = 0; i < 200; i++) {
+      try {
+        Thread.sleep(500);
+      } catch (InterruptedException e) {
+        // Restore the interrupted status
+        Thread.currentThread().interrupt();
+      }
+      try (RegionLocator locator = c.getRegionLocator(t.getName())) {
+        if (locator.getAllRegionLocations().size() > originalCount) {
+          return;
+        }
+      }
+    }
+    throw new Exception("Split did not increase the number of regions");
+  }
+
+  /*
+   * Take a snapshot of a table, add data, and verify that this only
+   * affects one table
+   * @param online - Whether the table is online or not during the snapshot
+   */
+  private void runTestSnapshotAppendIndependent(boolean online) throws Exception {
+    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+    Admin admin = UTIL.getHBaseAdmin();
+    final long startTime = System.currentTimeMillis();
+    final TableName localTableName =
+        TableName.valueOf(STRING_TABLE_NAME + startTime);
+
+    try (Table original = createTable(localTableName, TEST_FAM)) {
+      loadData(original, TEST_FAM);
+      final int origTableRowCount = countRows(original);
+
+      // Take a snapshot
+      final String snapshotNameAsString = "snapshot_" + localTableName;
+      byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+
+      SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+          snapshotNameAsString, rootDir, fs, online);
+
+      if (!online) {
+        tryDisable(admin, localTableName);
+      }
+      TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+      admin.cloneSnapshot(snapshotName, cloneTableName);
+
+      try (Table clonedTable = UTIL.getConnection().getTable(cloneTableName)) {
+
+        // Make sure that all the regions are available before starting
+        UTIL.waitUntilAllRegionsAssigned(cloneTableName);
+
+        final int clonedTableRowCount = countRows(clonedTable);
+
+        Assert.assertEquals(
+            "The line counts of original and cloned tables do not match after clone. ",
+            origTableRowCount, clonedTableRowCount);
+
+        // Attempt to add data to the test
+        final String rowKey = "new-row-" + System.currentTimeMillis();
+
+        Put p = new Put(Bytes.toBytes(rowKey));
+        p.addColumn(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
+        original.put(p);
+
+        // Verify that it is not present in the original table
+        Assert.assertEquals("The row count of the original table was not modified by the put",
+            origTableRowCount + 1, countRows(original));
+        Assert.assertEquals(
+            "The row count of the cloned table changed as a result of addition to the original",
+            clonedTableRowCount, countRows(clonedTable));
+
+        p = new Put(Bytes.toBytes(rowKey));
+        p.addColumn(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
+        clonedTable.put(p);
+
+        // Verify that the new family is not in the restored table's description
+        Assert.assertEquals(
+            "The row count of the original table was modified by the put to the clone",
+            origTableRowCount + 1, countRows(original));
+        Assert.assertEquals("The row count of the cloned table was not modified by the put",
+            clonedTableRowCount + 1, countRows(clonedTable));
+      }
+    }
+  }
+
+  /*
+   * Take a snapshot of a table, do a split, and verify that this only affects one table
+   * @param online - Whether the table is online or not during the snapshot
+   */
+  private void runTestRegionOperationsIndependent(boolean online) throws Exception {
+    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+    // Create a table
+    Admin admin = UTIL.getHBaseAdmin();
+    final long startTime = System.currentTimeMillis();
+    final TableName localTableName =
+        TableName.valueOf(STRING_TABLE_NAME + startTime);
+    Table original = createTable(localTableName, TEST_FAM);
+    loadData(original, TEST_FAM);
+    final int loadedTableCount = countRows(original);
+    System.out.println("Original table has: " + loadedTableCount + " rows");
+
+    final String snapshotNameAsString = "snapshot_" + localTableName;
+
+    // Create a snapshot
+    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+        snapshotNameAsString, rootDir, fs, online);
+
+    if (!online) {
+      tryDisable(admin, localTableName);
+    }
+
+    TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+
+    // Clone the snapshot
+    byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+    admin.cloneSnapshot(snapshotName, cloneTableName);
+
+    // Verify that region information is the same pre-split
+    ((ClusterConnection) UTIL.getConnection()).clearRegionCache();
+    List<HRegionInfo> originalTableHRegions = admin.getTableRegions(localTableName);
+
+    final int originalRegionCount = originalTableHRegions.size();
+    final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
+    Assert.assertEquals(
+        "The number of regions in the cloned table is different than in the original table.",
+        originalRegionCount, cloneTableRegionCount);
+
+    // Split a region on the parent table
+    admin.splitRegion(originalTableHRegions.get(0).getRegionName());
+    waitOnSplit(UTIL.getConnection(), original, originalRegionCount);
+
+    // Verify that the cloned table region is not split
+    final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
+    Assert.assertEquals(
+        "The number of regions in the cloned table changed though none of its regions were split.",
+        cloneTableRegionCount, cloneTableRegionCount2);
+  }
+
+  /*
+   * Take a snapshot of a table, add metadata, and verify that this only
+   * affects one table
+   * @param online - Whether the table is online or not during the snapshot
+   */
+  private void runTestSnapshotMetadataChangesIndependent(boolean online) throws Exception {
+    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+    // Create a table
+    Admin admin = UTIL.getHBaseAdmin();
+    final long startTime = System.currentTimeMillis();
+    final TableName localTableName =
+        TableName.valueOf(STRING_TABLE_NAME + startTime);
+    Table original = createTable(localTableName, TEST_FAM);
+    loadData(original, TEST_FAM);
+
+    final String snapshotNameAsString = "snapshot_" + localTableName;
+
+    // Create a snapshot
+    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+        snapshotNameAsString, rootDir, fs, online);
+
+    if (!online) {
+      tryDisable(admin, localTableName);
+    }
+
+    TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+
+    // Clone the snapshot
+    byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+    admin.cloneSnapshot(snapshotName, cloneTableName);
+
+    // Add a new column family to the original table
+    byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
+    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM_2);
+
+    tryDisable(admin, localTableName);
+    admin.addColumnFamily(localTableName, hcd);
+
+    // Verify that it is not in the snapshot
+    admin.enableTable(localTableName);
+    UTIL.waitTableAvailable(localTableName);
+
+    // get a description of the cloned table
+    // get a list of its families
+    // assert that the family is there
+    HTableDescriptor originalTableDescriptor = original.getTableDescriptor();
+    HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
+
+    Assert.assertTrue("The original family was not found. There is something wrong. ",
+        originalTableDescriptor.hasFamily(TEST_FAM));
+    Assert.assertTrue("The original family was not found in the clone. There is something wrong. ",
+        clonedTableDescriptor.hasFamily(TEST_FAM));
+
+    Assert.assertTrue("The new family was not found. ",
+        originalTableDescriptor.hasFamily(TEST_FAM_2));
+    Assert.assertTrue("The new family was not found. ",
+        !clonedTableDescriptor.hasFamily(TEST_FAM_2));
+  }
+
+  private void tryDisable(Admin admin, TableName localTableName) throws IOException {
+    int offlineRetry = 0;
+    while ( offlineRetry < 5 && admin.isTableEnabled(localTableName)) {
+      try {
+        admin.disableTable(localTableName);
+      } catch (IOException ioe) {
+        LOG.warn("Error disabling the table", ioe);
+      }
+      offlineRetry ++;
+    }
+  }
+
+  /*
+   * Take a snapshot of a table, add data, and verify that deleting the snapshot does not affect
+   * either table.
+   * @param online - Whether the table is online or not during the snapshot
+   */
+  private void runTestSnapshotDeleteIndependent(boolean online) throws Exception {
+    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+    final Admin admin = UTIL.getHBaseAdmin();
+    final long startTime = System.currentTimeMillis();
+    final TableName localTableName =
+        TableName.valueOf(STRING_TABLE_NAME + startTime);
+
+    try (Table original = createTable(localTableName, TEST_FAM)) {
+      loadData(original, TEST_FAM);
+    }
+
+    // Take a snapshot
+    final String snapshotNameAsString = "snapshot_" + localTableName;
+    byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+
+    SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+        snapshotNameAsString, rootDir, fs, online);
+
+    if (!online) {
+      tryDisable(admin, localTableName);
+    }
+
+    TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+    admin.cloneSnapshot(snapshotName, cloneTableName);
+
+    UTIL.waitUntilAllRegionsAssigned(cloneTableName);
+
+    // Ensure the original table does not reference the HFiles anymore
+    admin.majorCompact(localTableName);
+
+    // Deleting the snapshot used to break the cloned table by deleting in-use HFiles
+    admin.deleteSnapshot(snapshotName);
+
+    // Wait for cleaner run and DFS heartbeats so that anything that is deletable is fully deleted
+    do {
+      Thread.sleep(5000);
+    } while (!admin.listSnapshots(snapshotNameAsString).isEmpty());
+
+    try (Table original = UTIL.getConnection().getTable(localTableName)) {
+      try (Table clonedTable = UTIL.getConnection().getTable(cloneTableName)) {
+        // Verify that all regions of both tables are readable
+        final int origTableRowCount = countRows(original);
+        final int clonedTableRowCount = countRows(clonedTable);
+        Assert.assertEquals(origTableRowCount, clonedTableRowCount);
+      }
+    }
+  }
+
+  protected Table createTable(final TableName table, byte[] family) throws Exception {
+    Table t = UTIL.createTable(table, family);
+    // Wait for everything to be ready with the table
+    UTIL.waitUntilAllRegionsAssigned(table);
+
+    // At this point the table should be good to go.
+    return t;
+  }
+
+  protected void loadData(final Table table, byte[]... families) throws Exception {
+    UTIL.loadTable(table, families);
+  }
+
+  protected int countRows(final Table table, final byte[]... families) throws Exception {
+    return UTIL.countRows(table, families);
+  }
+}
\ No newline at end of file


[11/37] hbase git commit: HBASE-15302 Reenable the other tests disabled by HBASE-14678

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
new file mode 100644
index 0000000..c5728cf
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -0,0 +1,1799 @@
+/**
+ *
+
+ * 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.master;
+
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_wait_for_zk_delete;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_final_transition_failed;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_preempt_task;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_acquired;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_done;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_err;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_resigned;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SplitLogCounters;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.NonceGenerator;
+import org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
+import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
+import org.apache.hadoop.hbase.exceptions.OperationConflictException;
+import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, LargeTests.class})
+@SuppressWarnings("deprecation")
+public class TestDistributedLogSplitting {
+  private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
+  static {
+    // Uncomment the following line if more verbosity is needed for
+    // debugging (see HBASE-12285 for details).
+    //Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
+
+    // test ThreeRSAbort fails under hadoop2 (2.0.2-alpha) if shortcircuit-read (scr) is on. this
+    // turns it off for this test.  TODO: Figure out why scr breaks recovery.
+    System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
+
+  }
+
+  // Start a cluster with 2 masters and 6 regionservers
+  static final int NUM_MASTERS = 2;
+  static final int NUM_RS = 5;
+
+  MiniHBaseCluster cluster;
+  HMaster master;
+  Configuration conf;
+  static Configuration originalConf;
+  static HBaseTestingUtility TEST_UTIL;
+  static MiniDFSCluster dfsCluster;
+  static MiniZooKeeperCluster zkCluster;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility(HBaseConfiguration.create());
+    dfsCluster = TEST_UTIL.startMiniDFSCluster(1);
+    zkCluster = TEST_UTIL.startMiniZKCluster();
+    originalConf = TEST_UTIL.getConfiguration();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    TEST_UTIL.shutdownMiniZKCluster();
+    TEST_UTIL.shutdownMiniDFSCluster();
+    TEST_UTIL.shutdownMiniHBaseCluster();
+  }
+
+  private void startCluster(int num_rs) throws Exception {
+    SplitLogCounters.resetCounters();
+    LOG.info("Starting cluster");
+    conf.getLong("hbase.splitlog.max.resubmit", 0);
+    // Make the failure test faster
+    conf.setInt("zookeeper.recovery.retry", 0);
+    conf.setInt(HConstants.REGIONSERVER_INFO_PORT, -1);
+    conf.setFloat(HConstants.LOAD_BALANCER_SLOP_KEY, (float) 100.0); // no load balancing
+    conf.setInt("hbase.regionserver.wal.max.splitters", 3);
+    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
+    TEST_UTIL.shutdownMiniHBaseCluster();
+    TEST_UTIL = new HBaseTestingUtility(conf);
+    TEST_UTIL.setDFSCluster(dfsCluster);
+    TEST_UTIL.setZkCluster(zkCluster);
+    TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, num_rs);
+    cluster = TEST_UTIL.getHBaseCluster();
+    LOG.info("Waiting for active/ready master");
+    cluster.waitForActiveAndReadyMaster();
+    master = cluster.getMaster();
+    while (cluster.getLiveRegionServerThreads().size() < num_rs) {
+      Threads.sleep(10);
+    }
+  }
+
+  @Before
+  public void before() throws Exception {
+    // refresh configuration
+    conf = HBaseConfiguration.create(originalConf);
+  }
+
+  @After
+  public void after() throws Exception {
+    try {
+      if (TEST_UTIL.getHBaseCluster() != null) {
+        for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
+          mt.getMaster().abort("closing...", null);
+        }
+      }
+      TEST_UTIL.shutdownMiniHBaseCluster();
+    } finally {
+      TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
+      ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test (timeout=300000)
+  public void testRecoveredEdits() throws Exception {
+    LOG.info("testRecoveredEdits");
+    conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
+    startCluster(NUM_RS);
+
+    final int NUM_LOG_LINES = 1000;
+    final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+    FileSystem fs = master.getMasterFileSystem().getFileSystem();
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+
+    Path rootdir = FSUtils.getRootDir(conf);
+
+    Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+        "table", "family", 40);
+    try {
+      TableName table = t.getName();
+      List<HRegionInfo> regions = null;
+      HRegionServer hrs = null;
+      for (int i = 0; i < NUM_RS; i++) {
+        boolean foundRs = false;
+        hrs = rsts.get(i).getRegionServer();
+        regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        for (HRegionInfo region : regions) {
+          if (region.getTable().getNameAsString().equalsIgnoreCase("table")) {
+            foundRs = true;
+            break;
+          }
+        }
+        if (foundRs) break;
+      }
+      final Path logDir = new Path(rootdir, DefaultWALProvider.getWALDirectoryName(hrs
+          .getServerName().toString()));
+
+      LOG.info("#regions = " + regions.size());
+      Iterator<HRegionInfo> it = regions.iterator();
+      while (it.hasNext()) {
+        HRegionInfo region = it.next();
+        if (region.getTable().getNamespaceAsString()
+            .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
+          it.remove();
+        }
+      }
+
+      makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+      slm.splitLogDistributed(logDir);
+
+      int count = 0;
+      for (HRegionInfo hri : regions) {
+
+        Path tdir = FSUtils.getTableDir(rootdir, table);
+        Path editsdir =
+            WALSplitter.getRegionDirRecoveredEditsDir(
+                HRegion.getRegionDir(tdir, hri.getEncodedName()));
+        LOG.debug("checking edits dir " + editsdir);
+        FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
+          @Override
+          public boolean accept(Path p) {
+            if (WALSplitter.isSequenceIdFile(p)) {
+              return false;
+            }
+            return true;
+          }
+        });
+        assertTrue(
+            "edits dir should have more than a single file in it. instead has " + files.length,
+            files.length > 1);
+        for (int i = 0; i < files.length; i++) {
+          int c = countWAL(files[i].getPath(), fs, conf);
+          count += c;
+        }
+        LOG.info(count + " edits in " + files.length + " recovered edits files.");
+      }
+
+      // check that the log file is moved
+      assertFalse(fs.exists(logDir));
+
+      assertEquals(NUM_LOG_LINES, count);
+    } finally {
+      if (t != null) t.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testLogReplayWithNonMetaRSDown() throws Exception {
+    LOG.info("testLogReplayWithNonMetaRSDown");
+    conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      HRegionServer hrs = findRSToKill(false, "table");
+      List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+      makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+      // wait for abort completes
+      this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  private static class NonceGeneratorWithDups extends PerClientRandomNonceGenerator {
+    private boolean isDups = false;
+    private LinkedList<Long> nonces = new LinkedList<Long>();
+
+    public void startDups() {
+      isDups = true;
+    }
+
+    @Override
+    public long newNonce() {
+      long nonce = isDups ? nonces.removeFirst() : super.newNonce();
+      if (!isDups) {
+        nonces.add(nonce);
+      }
+      return nonce;
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testNonceRecovery() throws Exception {
+    LOG.info("testNonceRecovery");
+    final String TABLE_NAME = "table";
+    final String FAMILY_NAME = "family";
+    final int NUM_REGIONS_TO_CREATE = 40;
+
+    conf.setLong("hbase.regionserver.hlog.blocksize", 100*1024);
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    master.balanceSwitch(false);
+
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
+    NonceGeneratorWithDups ng = new NonceGeneratorWithDups();
+    NonceGenerator oldNg =
+        ConnectionUtils.injectNonceGeneratorForTesting(
+            (ClusterConnection)TEST_UTIL.getConnection(), ng);
+
+    try {
+      List<Increment> reqs = new ArrayList<Increment>();
+      for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
+        HRegionServer hrs = rst.getRegionServer();
+        List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        for (HRegionInfo hri : hris) {
+          if (TABLE_NAME.equalsIgnoreCase(hri.getTable().getNameAsString())) {
+            byte[] key = hri.getStartKey();
+            if (key == null || key.length == 0) {
+              key = Bytes.copy(hri.getEndKey());
+              --(key[key.length - 1]);
+            }
+            Increment incr = new Increment(key);
+            incr.addColumn(Bytes.toBytes(FAMILY_NAME), Bytes.toBytes("q"), 1);
+            ht.increment(incr);
+            reqs.add(incr);
+          }
+        }
+      }
+
+      HRegionServer hrs = findRSToKill(false, "table");
+      abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+      ng.startDups();
+      for (Increment incr : reqs) {
+        try {
+          ht.increment(incr);
+          fail("should have thrown");
+        } catch (OperationConflictException ope) {
+          LOG.debug("Caught as expected: " + ope.getMessage());
+        }
+      }
+    } finally {
+      ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)
+          TEST_UTIL.getConnection(), oldNg);
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testLogReplayWithMetaRSDown() throws Exception {
+    LOG.info("testRecoveredEditsReplayWithMetaRSDown");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      HRegionServer hrs = findRSToKill(true, "table");
+      List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+      makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+      this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  private void abortRSAndVerifyRecovery(HRegionServer hrs, Table ht, final ZooKeeperWatcher zkw,
+      final int numRegions, final int numofLines) throws Exception {
+
+    abortRSAndWaitForRecovery(hrs, zkw, numRegions);
+    assertEquals(numofLines, TEST_UTIL.countRows(ht));
+  }
+
+  private void abortRSAndWaitForRecovery(HRegionServer hrs, final ZooKeeperWatcher zkw,
+      final int numRegions) throws Exception {
+    final MiniHBaseCluster tmpCluster = this.cluster;
+
+    // abort RS
+    LOG.info("Aborting region server: " + hrs.getServerName());
+    hrs.abort("testing");
+
+    // wait for abort completes
+    TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return (tmpCluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+      }
+    });
+
+    // wait for regions come online
+    TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return (HBaseTestingUtility.getAllOnlineRegions(tmpCluster).size()
+            >= (numRegions + 1));
+      }
+    });
+
+    // wait for all regions are fully recovered
+    TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+            zkw.recoveringRegionsZNode, false);
+        return (recoveringRegions != null && recoveringRegions.size() == 0);
+      }
+    });
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testMasterStartsUpWithLogSplittingWork() throws Exception {
+    LOG.info("testMasterStartsUpWithLogSplittingWork");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
+    conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
+    startCluster(NUM_RS);
+
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      HRegionServer hrs = findRSToKill(false, "table");
+      List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+      makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+      // abort master
+      abortMaster(cluster);
+
+      // abort RS
+      LOG.info("Aborting region server: " + hrs.getServerName());
+      hrs.abort("testing");
+
+      // wait for abort completes
+      TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+        }
+      });
+
+      Thread.sleep(2000);
+      LOG.info("Current Open Regions:"
+          + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+      // wait for abort completes
+      TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+              >= (NUM_REGIONS_TO_CREATE + 1));
+        }
+      });
+
+      LOG.info("Current Open Regions After Master Node Starts Up:"
+          + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+      assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testMasterStartsUpWithLogReplayWork() throws Exception {
+    LOG.info("testMasterStartsUpWithLogReplayWork");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
+    startCluster(NUM_RS);
+
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      HRegionServer hrs = findRSToKill(false, "table");
+      List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+      makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+      // abort master
+      abortMaster(cluster);
+
+      // abort RS
+      LOG.info("Aborting region server: " + hrs.getServerName());
+      hrs.abort("testing");
+
+      // wait for the RS dies
+      TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+        }
+      });
+
+      Thread.sleep(2000);
+      LOG.info("Current Open Regions:" + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+      // wait for all regions are fully recovered
+      TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+              zkw.recoveringRegionsZNode, false);
+          boolean done = recoveringRegions != null && recoveringRegions.size() == 0;
+          if (!done) {
+            LOG.info("Recovering regions: " + recoveringRegions);
+          }
+          return done;
+        }
+      });
+
+      LOG.info("Current Open Regions After Master Node Starts Up:"
+          + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+      assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testLogReplayTwoSequentialRSDown() throws Exception {
+    LOG.info("testRecoveredEditsReplayTwoSequentialRSDown");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      List<HRegionInfo> regions = null;
+      HRegionServer hrs1 = findRSToKill(false, "table");
+      regions = ProtobufUtil.getOnlineRegions(hrs1.getRSRpcServices());
+
+      makeWAL(hrs1, regions, "table", "family", NUM_LOG_LINES, 100);
+
+      // abort RS1
+      LOG.info("Aborting region server: " + hrs1.getServerName());
+      hrs1.abort("testing");
+
+      // wait for abort completes
+      TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+        }
+      });
+
+      // wait for regions come online
+      TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+              >= (NUM_REGIONS_TO_CREATE + 1));
+        }
+      });
+
+      // sleep a little bit in order to interrupt recovering in the middle
+      Thread.sleep(300);
+      // abort second region server
+      rsts = cluster.getLiveRegionServerThreads();
+      HRegionServer hrs2 = rsts.get(0).getRegionServer();
+      LOG.info("Aborting one more region server: " + hrs2.getServerName());
+      hrs2.abort("testing");
+
+      // wait for abort completes
+      TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 2));
+        }
+      });
+
+      // wait for regions come online
+      TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+              >= (NUM_REGIONS_TO_CREATE + 1));
+        }
+      });
+
+      // wait for all regions are fully recovered
+      TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+              zkw.recoveringRegionsZNode, false);
+          return (recoveringRegions != null && recoveringRegions.size() == 0);
+        }
+      });
+
+      assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testMarkRegionsRecoveringInZK() throws Exception {
+    LOG.info("testMarkRegionsRecoveringInZK");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    master.balanceSwitch(false);
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = master.getZooKeeper();
+    Table ht = installTable(zkw, "table", "family", 40);
+    try {
+      final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+
+      Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
+      HRegionInfo region = null;
+      HRegionServer hrs = null;
+      ServerName firstFailedServer = null;
+      ServerName secondFailedServer = null;
+      for (int i = 0; i < NUM_RS; i++) {
+        hrs = rsts.get(i).getRegionServer();
+        List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        if (regions.isEmpty()) continue;
+        region = regions.get(0);
+        regionSet.add(region);
+        firstFailedServer = hrs.getServerName();
+        secondFailedServer = rsts.get((i + 1) % NUM_RS).getRegionServer().getServerName();
+        break;
+      }
+
+      slm.markRegionsRecovering(firstFailedServer, regionSet);
+      slm.markRegionsRecovering(secondFailedServer, regionSet);
+
+      List<String> recoveringRegions = ZKUtil.listChildrenNoWatch(zkw,
+          ZKUtil.joinZNode(zkw.recoveringRegionsZNode, region.getEncodedName()));
+
+      assertEquals(recoveringRegions.size(), 2);
+
+      // wait for splitLogWorker to mark them up because there is no WAL files recorded in ZK
+      final HRegionServer tmphrs = hrs;
+      TEST_UTIL.waitFor(60000, 1000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (tmphrs.getRecoveringRegions().size() == 0);
+        }
+      });
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testReplayCmd() throws Exception {
+    LOG.info("testReplayCmd");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      List<HRegionInfo> regions = null;
+      HRegionServer hrs = null;
+      for (int i = 0; i < NUM_RS; i++) {
+        boolean isCarryingMeta = false;
+        hrs = rsts.get(i).getRegionServer();
+        regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        for (HRegionInfo region : regions) {
+          if (region.isMetaRegion()) {
+            isCarryingMeta = true;
+            break;
+          }
+        }
+        if (isCarryingMeta) {
+          continue;
+        }
+        if (regions.size() > 0) break;
+      }
+
+      this.prepareData(ht, Bytes.toBytes("family"), Bytes.toBytes("c1"));
+      String originalCheckSum = TEST_UTIL.checksumRows(ht);
+
+      // abort RA and trigger replay
+      abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+
+      assertEquals("Data should remain after reopening of regions", originalCheckSum,
+          TEST_UTIL.checksumRows(ht));
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testLogReplayForDisablingTable() throws Exception {
+    LOG.info("testLogReplayForDisablingTable");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table disablingHT = installTable(zkw, "disableTable", "family", NUM_REGIONS_TO_CREATE);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE, NUM_REGIONS_TO_CREATE);
+    try {
+      // turn off load balancing to prevent regions from moving around otherwise
+      // they will consume recovered.edits
+      master.balanceSwitch(false);
+
+      List<HRegionInfo> regions = null;
+      HRegionServer hrs = null;
+      boolean hasRegionsForBothTables = false;
+      String tableName = null;
+      for (int i = 0; i < NUM_RS; i++) {
+        tableName = null;
+        hasRegionsForBothTables = false;
+        boolean isCarryingSystem = false;
+        hrs = rsts.get(i).getRegionServer();
+        regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        for (HRegionInfo region : regions) {
+          if (region.getTable().isSystemTable()) {
+            isCarryingSystem = true;
+            break;
+          }
+          if (tableName != null &&
+              !tableName.equalsIgnoreCase(region.getTable().getNameAsString())) {
+            // make sure that we find a RS has online regions for both "table" and "disableTable"
+            hasRegionsForBothTables = true;
+            break;
+          } else if (tableName == null) {
+            tableName = region.getTable().getNameAsString();
+          }
+        }
+        if (isCarryingSystem) {
+          continue;
+        }
+        if (hasRegionsForBothTables) {
+          break;
+        }
+      }
+
+      // make sure we found a good RS
+      Assert.assertTrue(hasRegionsForBothTables);
+
+      LOG.info("#regions = " + regions.size());
+      Iterator<HRegionInfo> it = regions.iterator();
+      while (it.hasNext()) {
+        HRegionInfo region = it.next();
+        if (region.isMetaTable()) {
+          it.remove();
+        }
+      }
+      makeWAL(hrs, regions, "disableTable", "family", NUM_LOG_LINES, 100, false);
+      makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+      LOG.info("Disabling table\n");
+      TEST_UTIL.getHBaseAdmin().disableTable(TableName.valueOf("disableTable"));
+      TEST_UTIL.waitTableDisabled(TableName.valueOf("disableTable").getName());
+
+      // abort RS
+      LOG.info("Aborting region server: " + hrs.getServerName());
+      hrs.abort("testing");
+
+      // wait for abort completes
+      TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+        }
+      });
+
+      // wait for regions come online
+      TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+              >= (NUM_REGIONS_TO_CREATE + 1));
+        }
+      });
+
+      // wait for all regions are fully recovered
+      TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+              zkw.recoveringRegionsZNode, false);
+          ServerManager serverManager = master.getServerManager();
+          return (!serverManager.areDeadServersInProgress() &&
+              recoveringRegions != null && recoveringRegions.size() == 0);
+        }
+      });
+
+      int count = 0;
+      FileSystem fs = master.getMasterFileSystem().getFileSystem();
+      Path rootdir = FSUtils.getRootDir(conf);
+      Path tdir = FSUtils.getTableDir(rootdir, TableName.valueOf("disableTable"));
+      for (HRegionInfo hri : regions) {
+        Path editsdir =
+            WALSplitter.getRegionDirRecoveredEditsDir(
+                HRegion.getRegionDir(tdir, hri.getEncodedName()));
+        LOG.debug("checking edits dir " + editsdir);
+        if(!fs.exists(editsdir)) continue;
+        FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
+          @Override
+          public boolean accept(Path p) {
+            if (WALSplitter.isSequenceIdFile(p)) {
+              return false;
+            }
+            return true;
+          }
+        });
+        if(files != null) {
+          for(FileStatus file : files) {
+            int c = countWAL(file.getPath(), fs, conf);
+            count += c;
+            LOG.info(c + " edits in " + file.getPath());
+          }
+        }
+      }
+
+      LOG.info("Verify edits in recovered.edits files");
+      assertEquals(NUM_LOG_LINES, count);
+      LOG.info("Verify replayed edits");
+      assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+
+      // clean up
+      for (HRegionInfo hri : regions) {
+        Path editsdir =
+            WALSplitter.getRegionDirRecoveredEditsDir(
+                HRegion.getRegionDir(tdir, hri.getEncodedName()));
+        fs.delete(editsdir, true);
+      }
+      disablingHT.close();
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testDisallowWritesInRecovering() throws Exception {
+    LOG.info("testDisallowWritesInRecovering");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
+    conf.setBoolean(HConstants.DISALLOW_WRITES_IN_RECOVERING, true);
+    startCluster(NUM_RS);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+
+      Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
+      HRegionInfo region = null;
+      HRegionServer hrs = null;
+      HRegionServer dstRS = null;
+      for (int i = 0; i < NUM_RS; i++) {
+        hrs = rsts.get(i).getRegionServer();
+        List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        if (regions.isEmpty()) continue;
+        region = regions.get(0);
+        regionSet.add(region);
+        dstRS = rsts.get((i+1) % NUM_RS).getRegionServer();
+        break;
+      }
+
+      slm.markRegionsRecovering(hrs.getServerName(), regionSet);
+      // move region in order for the region opened in recovering state
+      final HRegionInfo hri = region;
+      final HRegionServer tmpRS = dstRS;
+      TEST_UTIL.getHBaseAdmin().move(region.getEncodedNameAsBytes(),
+          Bytes.toBytes(dstRS.getServerName().getServerName()));
+      // wait for region move completes
+      final RegionStates regionStates =
+          TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+      TEST_UTIL.waitFor(45000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          ServerName sn = regionStates.getRegionServerOfRegion(hri);
+          return (sn != null && sn.equals(tmpRS.getServerName()));
+        }
+      });
+
+      try {
+        byte[] key = region.getStartKey();
+        if (key == null || key.length == 0) {
+          key = new byte[] { 0, 0, 0, 0, 1 };
+        }
+        Put put = new Put(key);
+        put.addColumn(Bytes.toBytes("family"), Bytes.toBytes("c1"), new byte[]{'b'});
+        ht.put(put);
+      } catch (IOException ioe) {
+        Assert.assertTrue(ioe instanceof RetriesExhaustedWithDetailsException);
+        RetriesExhaustedWithDetailsException re = (RetriesExhaustedWithDetailsException) ioe;
+        boolean foundRegionInRecoveryException = false;
+        for (Throwable t : re.getCauses()) {
+          if (t instanceof RegionInRecoveryException) {
+            foundRegionInRecoveryException = true;
+            break;
+          }
+        }
+        Assert.assertTrue(
+            "No RegionInRecoveryException. Following exceptions returned=" + re.getCauses(),
+            foundRegionInRecoveryException);
+      }
+    } finally {
+      if (ht != null) ht.close();
+      if (ht != null) zkw.close();
+    }
+  }
+
+  /**
+   * The original intention of this test was to force an abort of a region
+   * server and to make sure that the failure path in the region servers is
+   * properly evaluated. But it is difficult to ensure that the region server
+   * doesn't finish the log splitting before it aborts. Also now, there is
+   * this code path where the master will preempt the region server when master
+   * detects that the region server has aborted.
+   * @throws Exception
+   */
+  @Ignore ("Disabled because flakey") @Test (timeout=300000)
+  public void testWorkerAbort() throws Exception {
+    LOG.info("testWorkerAbort");
+    startCluster(3);
+    final int NUM_LOG_LINES = 10000;
+    final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+    FileSystem fs = master.getMasterFileSystem().getFileSystem();
+
+    final List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    HRegionServer hrs = findRSToKill(false, "table");
+    Path rootdir = FSUtils.getRootDir(conf);
+    final Path logDir = new Path(rootdir,
+        DefaultWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
+
+    Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+        "table", "family", 40);
+    try {
+      makeWAL(hrs, ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()),
+          "table", "family", NUM_LOG_LINES, 100);
+
+      new Thread() {
+        @Override
+        public void run() {
+          waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
+          for (RegionServerThread rst : rsts) {
+            rst.getRegionServer().abort("testing");
+            break;
+          }
+        }
+      }.start();
+      // slm.splitLogDistributed(logDir);
+      FileStatus[] logfiles = fs.listStatus(logDir);
+      TaskBatch batch = new TaskBatch();
+      slm.enqueueSplitTask(logfiles[0].getPath().toString(), batch);
+      //waitForCounter but for one of the 2 counters
+      long curt = System.currentTimeMillis();
+      long waitTime = 80000;
+      long endt = curt + waitTime;
+      while (curt < endt) {
+        if ((tot_wkr_task_resigned.get() + tot_wkr_task_err.get() +
+            tot_wkr_final_transition_failed.get() + tot_wkr_task_done.get() +
+            tot_wkr_preempt_task.get()) == 0) {
+          Thread.yield();
+          curt = System.currentTimeMillis();
+        } else {
+          assertTrue(1 <= (tot_wkr_task_resigned.get() + tot_wkr_task_err.get() +
+              tot_wkr_final_transition_failed.get() + tot_wkr_task_done.get() +
+              tot_wkr_preempt_task.get()));
+          return;
+        }
+      }
+      fail("none of the following counters went up in " + waitTime +
+          " milliseconds - " +
+          "tot_wkr_task_resigned, tot_wkr_task_err, " +
+          "tot_wkr_final_transition_failed, tot_wkr_task_done, " +
+          "tot_wkr_preempt_task");
+    } finally {
+      if (t != null) t.close();
+    }
+  }
+
+  @Test (timeout=300000)
+  public void testThreeRSAbort() throws Exception {
+    LOG.info("testThreeRSAbort");
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_ROWS_PER_REGION = 100;
+
+    startCluster(NUM_RS); // NUM_RS=6.
+
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+        "distributed log splitting test", null);
+
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      populateDataInTable(NUM_ROWS_PER_REGION, "family");
+
+
+      List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+      assertEquals(NUM_RS, rsts.size());
+      rsts.get(0).getRegionServer().abort("testing");
+      rsts.get(1).getRegionServer().abort("testing");
+      rsts.get(2).getRegionServer().abort("testing");
+
+      long start = EnvironmentEdgeManager.currentTime();
+      while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
+        if (EnvironmentEdgeManager.currentTime() - start > 60000) {
+          assertTrue(false);
+        }
+        Thread.sleep(200);
+      }
+
+      start = EnvironmentEdgeManager.currentTime();
+      while (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+          < (NUM_REGIONS_TO_CREATE + 1)) {
+        if (EnvironmentEdgeManager.currentTime() - start > 60000) {
+          assertTrue("Timedout", false);
+        }
+        Thread.sleep(200);
+      }
+
+      // wait for all regions are fully recovered
+      TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+              zkw.recoveringRegionsZNode, false);
+          return (recoveringRegions != null && recoveringRegions.size() == 0);
+        }
+      });
+
+      assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
+          TEST_UTIL.countRows(ht));
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+
+
+  @Test(timeout=30000)
+  public void testDelayedDeleteOnFailure() throws Exception {
+    LOG.info("testDelayedDeleteOnFailure");
+    startCluster(1);
+    final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+    final FileSystem fs = master.getMasterFileSystem().getFileSystem();
+    final Path logDir = new Path(FSUtils.getRootDir(conf), "x");
+    fs.mkdirs(logDir);
+    ExecutorService executor = null;
+    try {
+      final Path corruptedLogFile = new Path(logDir, "x");
+      FSDataOutputStream out;
+      out = fs.create(corruptedLogFile);
+      out.write(0);
+      out.write(Bytes.toBytes("corrupted bytes"));
+      out.close();
+      ZKSplitLogManagerCoordination coordination =
+          (ZKSplitLogManagerCoordination) ((BaseCoordinatedStateManager) master
+              .getCoordinatedStateManager()).getSplitLogManagerCoordination();
+      coordination.setIgnoreDeleteForTesting(true);
+      executor = Executors.newSingleThreadExecutor();
+      Runnable runnable = new Runnable() {
+        @Override
+        public void run() {
+          try {
+            // since the logDir is a fake, corrupted one, so the split log worker
+            // will finish it quickly with error, and this call will fail and throw
+            // an IOException.
+            slm.splitLogDistributed(logDir);
+          } catch (IOException ioe) {
+            try {
+              assertTrue(fs.exists(corruptedLogFile));
+              // this call will block waiting for the task to be removed from the
+              // tasks map which is not going to happen since ignoreZKDeleteForTesting
+              // is set to true, until it is interrupted.
+              slm.splitLogDistributed(logDir);
+            } catch (IOException e) {
+              assertTrue(Thread.currentThread().isInterrupted());
+              return;
+            }
+            fail("did not get the expected IOException from the 2nd call");
+          }
+          fail("did not get the expected IOException from the 1st call");
+        }
+      };
+      Future<?> result = executor.submit(runnable);
+      try {
+        result.get(2000, TimeUnit.MILLISECONDS);
+      } catch (TimeoutException te) {
+        // it is ok, expected.
+      }
+      waitForCounter(tot_mgr_wait_for_zk_delete, 0, 1, 10000);
+      executor.shutdownNow();
+      executor = null;
+
+      // make sure the runnable is finished with no exception thrown.
+      result.get();
+    } finally {
+      if (executor != null) {
+        // interrupt the thread in case the test fails in the middle.
+        // it has no effect if the thread is already terminated.
+        executor.shutdownNow();
+      }
+      fs.delete(logDir, true);
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testMetaRecoveryInZK() throws Exception {
+    LOG.info("testMetaRecoveryInZK");
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+
+    // only testing meta recovery in ZK operation
+    HRegionServer hrs = findRSToKill(true, null);
+    List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+
+    LOG.info("#regions = " + regions.size());
+    Set<HRegionInfo> tmpRegions = new HashSet<HRegionInfo>();
+    tmpRegions.add(HRegionInfo.FIRST_META_REGIONINFO);
+    master.getMasterFileSystem().prepareLogReplay(hrs.getServerName(), tmpRegions);
+    Set<HRegionInfo> userRegionSet = new HashSet<HRegionInfo>();
+    userRegionSet.addAll(regions);
+    master.getMasterFileSystem().prepareLogReplay(hrs.getServerName(), userRegionSet);
+    boolean isMetaRegionInRecovery = false;
+    List<String> recoveringRegions =
+        zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false);
+    for (String curEncodedRegionName : recoveringRegions) {
+      if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+        isMetaRegionInRecovery = true;
+        break;
+      }
+    }
+    assertTrue(isMetaRegionInRecovery);
+
+    master.getMasterFileSystem().splitMetaLog(hrs.getServerName());
+
+    isMetaRegionInRecovery = false;
+    recoveringRegions =
+        zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false);
+    for (String curEncodedRegionName : recoveringRegions) {
+      if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+        isMetaRegionInRecovery = true;
+        break;
+      }
+    }
+    // meta region should be recovered
+    assertFalse(isMetaRegionInRecovery);
+    zkw.close();
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testSameVersionUpdatesRecovery() throws Exception {
+    LOG.info("testSameVersionUpdatesRecovery");
+    conf.setLong("hbase.regionserver.hlog.blocksize", 15 * 1024);
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    startCluster(NUM_RS);
+    final AtomicLong sequenceId = new AtomicLong(100);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 1000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      List<HRegionInfo> regions = null;
+      HRegionServer hrs = null;
+      for (int i = 0; i < NUM_RS; i++) {
+        boolean isCarryingMeta = false;
+        hrs = rsts.get(i).getRegionServer();
+        regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        for (HRegionInfo region : regions) {
+          if (region.isMetaRegion()) {
+            isCarryingMeta = true;
+            break;
+          }
+        }
+        if (isCarryingMeta) {
+          continue;
+        }
+        break;
+      }
+
+      LOG.info("#regions = " + regions.size());
+      Iterator<HRegionInfo> it = regions.iterator();
+      while (it.hasNext()) {
+        HRegionInfo region = it.next();
+        if (region.isMetaTable()
+            || region.getEncodedName().equals(
+            HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+          it.remove();
+        }
+      }
+      if (regions.size() == 0) return;
+      HRegionInfo curRegionInfo = regions.get(0);
+      byte[] startRow = curRegionInfo.getStartKey();
+      if (startRow == null || startRow.length == 0) {
+        startRow = new byte[] { 0, 0, 0, 0, 1 };
+      }
+      byte[] row = Bytes.incrementBytes(startRow, 1);
+      // use last 5 bytes because HBaseTestingUtility.createMultiRegions use 5 bytes key
+      row = Arrays.copyOfRange(row, 3, 8);
+      long value = 0;
+      TableName tableName = TableName.valueOf("table");
+      byte[] family = Bytes.toBytes("family");
+      byte[] qualifier = Bytes.toBytes("c1");
+      long timeStamp = System.currentTimeMillis();
+      HTableDescriptor htd = new HTableDescriptor(tableName);
+      htd.addFamily(new HColumnDescriptor(family));
+      final WAL wal = hrs.getWAL(curRegionInfo);
+      for (int i = 0; i < NUM_LOG_LINES; i += 1) {
+        WALEdit e = new WALEdit();
+        value++;
+        e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
+        wal.append(htd, curRegionInfo,
+            new HLogKey(curRegionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis()),
+            e, true);
+      }
+      wal.sync();
+      wal.shutdown();
+
+      // wait for abort completes
+      this.abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+
+      // verify we got the last value
+      LOG.info("Verification Starts...");
+      Get g = new Get(row);
+      Result r = ht.get(g);
+      long theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+      assertEquals(value, theStoredVal);
+
+      // after flush
+      LOG.info("Verification after flush...");
+      TEST_UTIL.getHBaseAdmin().flush(tableName);
+      r = ht.get(g);
+      theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+      assertEquals(value, theStoredVal);
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+  public void testSameVersionUpdatesRecoveryWithCompaction() throws Exception {
+    LOG.info("testSameVersionUpdatesRecoveryWithWrites");
+    conf.setLong("hbase.regionserver.hlog.blocksize", 15 * 1024);
+    conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+    conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 30 * 1024);
+    conf.setInt("hbase.hstore.compactionThreshold", 3);
+    startCluster(NUM_RS);
+    final AtomicLong sequenceId = new AtomicLong(100);
+    final int NUM_REGIONS_TO_CREATE = 40;
+    final int NUM_LOG_LINES = 2000;
+    // turn off load balancing to prevent regions from moving around otherwise
+    // they will consume recovered.edits
+    master.balanceSwitch(false);
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    try {
+      List<HRegionInfo> regions = null;
+      HRegionServer hrs = null;
+      for (int i = 0; i < NUM_RS; i++) {
+        boolean isCarryingMeta = false;
+        hrs = rsts.get(i).getRegionServer();
+        regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+        for (HRegionInfo region : regions) {
+          if (region.isMetaRegion()) {
+            isCarryingMeta = true;
+            break;
+          }
+        }
+        if (isCarryingMeta) {
+          continue;
+        }
+        break;
+      }
+
+      LOG.info("#regions = " + regions.size());
+      Iterator<HRegionInfo> it = regions.iterator();
+      while (it.hasNext()) {
+        HRegionInfo region = it.next();
+        if (region.isMetaTable()
+            || region.getEncodedName().equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+          it.remove();
+        }
+      }
+      if (regions.size() == 0) return;
+      HRegionInfo curRegionInfo = regions.get(0);
+      byte[] startRow = curRegionInfo.getStartKey();
+      if (startRow == null || startRow.length == 0) {
+        startRow = new byte[] { 0, 0, 0, 0, 1 };
+      }
+      byte[] row = Bytes.incrementBytes(startRow, 1);
+      // use last 5 bytes because HBaseTestingUtility.createMultiRegions use 5 bytes key
+      row = Arrays.copyOfRange(row, 3, 8);
+      long value = 0;
+      final TableName tableName = TableName.valueOf("table");
+      byte[] family = Bytes.toBytes("family");
+      byte[] qualifier = Bytes.toBytes("c1");
+      long timeStamp = System.currentTimeMillis();
+      HTableDescriptor htd = new HTableDescriptor(tableName);
+      htd.addFamily(new HColumnDescriptor(family));
+      final WAL wal = hrs.getWAL(curRegionInfo);
+      for (int i = 0; i < NUM_LOG_LINES; i += 1) {
+        WALEdit e = new WALEdit();
+        value++;
+        e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
+        wal.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(),
+            tableName, System.currentTimeMillis()), e, true);
+      }
+      wal.sync();
+      wal.shutdown();
+
+      // wait for abort completes
+      this.abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+
+      // verify we got the last value
+      LOG.info("Verification Starts...");
+      Get g = new Get(row);
+      Result r = ht.get(g);
+      long theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+      assertEquals(value, theStoredVal);
+
+      // after flush & compaction
+      LOG.info("Verification after flush...");
+      TEST_UTIL.getHBaseAdmin().flush(tableName);
+      TEST_UTIL.getHBaseAdmin().compact(tableName);
+
+      // wait for compaction completes
+      TEST_UTIL.waitFor(30000, 200, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return (TEST_UTIL.getHBaseAdmin().getCompactionState(tableName) == CompactionState.NONE);
+        }
+      });
+
+      r = ht.get(g);
+      theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+      assertEquals(value, theStoredVal);
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testReadWriteSeqIdFiles() throws Exception {
+    LOG.info("testReadWriteSeqIdFiles");
+    startCluster(2);
+    final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+    Table ht = installTable(zkw, "table", "family", 10);
+    try {
+      FileSystem fs = master.getMasterFileSystem().getFileSystem();
+      Path tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), TableName.valueOf("table"));
+      List<Path> regionDirs = FSUtils.getRegionDirs(fs, tableDir);
+      long newSeqId = WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 1L, 1000L);
+      WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0) , 1L, 1000L);
+      assertEquals(newSeqId + 2000,
+          WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 3L, 1000L));
+
+      Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regionDirs.get(0));
+      FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
+        @Override
+        public boolean accept(Path p) {
+          return WALSplitter.isSequenceIdFile(p);
+        }
+      });
+      // only one seqid file should exist
+      assertEquals(1, files.length);
+
+      // verify all seqId files aren't treated as recovered.edits files
+      NavigableSet<Path> recoveredEdits =
+          WALSplitter.getSplitEditFilesSorted(fs, regionDirs.get(0));
+      assertEquals(0, recoveredEdits.size());
+    } finally {
+      if (ht != null) ht.close();
+      if (zkw != null) zkw.close();
+    }
+  }
+
+  Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
+    return installTable(zkw, tname, fname, nrs, 0);
+  }
+
+  Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
+      int existingRegions) throws Exception {
+    // Create a table with regions
+    TableName table = TableName.valueOf(tname);
+    byte [] family = Bytes.toBytes(fname);
+    LOG.info("Creating table with " + nrs + " regions");
+    Table ht = TEST_UTIL.createMultiRegionTable(table, family, nrs);
+    int numRegions = -1;
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
+      numRegions = r.getStartKeys().length;
+    }
+    assertEquals(nrs, numRegions);
+    LOG.info("Waiting for no more RIT\n");
+    blockUntilNoRIT(zkw, master);
+    // disable-enable cycle to get rid of table's dead regions left behind
+    // by createMultiRegions
+    LOG.debug("Disabling table\n");
+    TEST_UTIL.getHBaseAdmin().disableTable(table);
+    LOG.debug("Waiting for no more RIT\n");
+    blockUntilNoRIT(zkw, master);
+    NavigableSet<String> regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
+    LOG.debug("Verifying only catalog and namespace regions are assigned\n");
+    if (regions.size() != 2) {
+      for (String oregion : regions)
+        LOG.debug("Region still online: " + oregion);
+    }
+    assertEquals(2 + existingRegions, regions.size());
+    LOG.debug("Enabling table\n");
+    TEST_UTIL.getHBaseAdmin().enableTable(table);
+    LOG.debug("Waiting for no more RIT\n");
+    blockUntilNoRIT(zkw, master);
+    LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
+    regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
+    assertEquals(numRegions + 2 + existingRegions, regions.size());
+    return ht;
+  }
+
+  void populateDataInTable(int nrows, String fname) throws Exception {
+    byte [] family = Bytes.toBytes(fname);
+
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    assertEquals(NUM_RS, rsts.size());
+
+    for (RegionServerThread rst : rsts) {
+      HRegionServer hrs = rst.getRegionServer();
+      List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+      for (HRegionInfo hri : hris) {
+        if (hri.getTable().isSystemTable()) {
+          continue;
+        }
+        LOG.debug("adding data to rs = " + rst.getName() +
+            " region = "+ hri.getRegionNameAsString());
+        Region region = hrs.getOnlineRegion(hri.getRegionName());
+        assertTrue(region != null);
+        putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
+      }
+    }
+
+    for (MasterThread mt : cluster.getLiveMasterThreads()) {
+      HRegionServer hrs = mt.getMaster();
+      List<HRegionInfo> hris;
+      try {
+        hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+      } catch (ServerNotRunningYetException e) {
+        // It's ok: this master may be a backup. Ignored.
+        continue;
+      }
+      for (HRegionInfo hri : hris) {
+        if (hri.getTable().isSystemTable()) {
+          continue;
+        }
+        LOG.debug("adding data to rs = " + mt.getName() +
+            " region = "+ hri.getRegionNameAsString());
+        Region region = hrs.getOnlineRegion(hri.getRegionName());
+        assertTrue(region != null);
+        putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
+      }
+    }
+  }
+
+  public void makeWAL(HRegionServer hrs, List<HRegionInfo> regions, String tname, String fname,
+      int num_edits, int edit_size) throws IOException {
+    makeWAL(hrs, regions, tname, fname, num_edits, edit_size, true);
+  }
+
+  public void makeWAL(HRegionServer hrs, List<HRegionInfo> regions, String tname, String fname,
+      int num_edits, int edit_size, boolean cleanShutdown) throws IOException {
+    TableName fullTName = TableName.valueOf(tname);
+    // remove root and meta region
+    regions.remove(HRegionInfo.FIRST_META_REGIONINFO);
+    // using one sequenceId for edits across all regions is ok.
+    final AtomicLong sequenceId = new AtomicLong(10);
+
+
+    for(Iterator<HRegionInfo> iter = regions.iterator(); iter.hasNext(); ) {
+      HRegionInfo regionInfo = iter.next();
+      if(regionInfo.getTable().isSystemTable()) {
+        iter.remove();
+      }
+    }
+    HTableDescriptor htd = new HTableDescriptor(fullTName);
+    byte[] family = Bytes.toBytes(fname);
+    htd.addFamily(new HColumnDescriptor(family));
+    byte[] value = new byte[edit_size];
+
+    List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
+    for (HRegionInfo region : regions) {
+      if (!region.getTable().getNameAsString().equalsIgnoreCase(tname)) {
+        continue;
+      }
+      hris.add(region);
+    }
+    LOG.info("Creating wal edits across " + hris.size() + " regions.");
+    for (int i = 0; i < edit_size; i++) {
+      value[i] = (byte) ('a' + (i % 26));
+    }
+    int n = hris.size();
+    int[] counts = new int[n];
+    // sync every ~30k to line up with desired wal rolls
+    final int syncEvery = 30 * 1024 / edit_size;
+    if (n > 0) {
+      for (int i = 0; i < num_edits; i += 1) {
+        WALEdit e = new WALEdit();
+        HRegionInfo curRegionInfo = hris.get(i % n);
+        final WAL log = hrs.getWAL(curRegionInfo);
+        byte[] startRow = curRegionInfo.getStartKey();
+        if (startRow == null || startRow.length == 0) {
+          startRow = new byte[] { 0, 0, 0, 0, 1 };
+        }
+        byte[] row = Bytes.incrementBytes(startRow, counts[i % n]);
+        row = Arrays.copyOfRange(row, 3, 8); // use last 5 bytes because
+        // HBaseTestingUtility.createMultiRegions use 5 bytes
+        // key
+        byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
+        e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
+        log.append(htd, curRegionInfo,
+            new HLogKey(curRegionInfo.getEncodedNameAsBytes(), fullTName,
+                System.currentTimeMillis()), e, true);
+        if (0 == i % syncEvery) {
+          log.sync();
+        }
+        counts[i % n] += 1;
+      }
+    }
+    // done as two passes because the regions might share logs. shutdown is idempotent, but sync
+    // will cause errors if done after.
+    for (HRegionInfo info : hris) {
+      final WAL log = hrs.getWAL(info);
+      log.sync();
+    }
+    if (cleanShutdown) {
+      for (HRegionInfo info : hris) {
+        final WAL log = hrs.getWAL(info);
+        log.shutdown();
+      }
+    }
+    for (int i = 0; i < n; i++) {
+      LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits");
+    }
+    return;
+  }
+
+  private int countWAL(Path log, FileSystem fs, Configuration conf)
+      throws IOException {
+    int count = 0;
+    WAL.Reader in = WALFactory.createReader(fs, log, conf);
+    try {
+      WAL.Entry e;
+      while ((e = in.next()) != null) {
+        if (!WALEdit.isMetaEditFamily(e.getEdit().getCells().get(0))) {
+          count++;
+        }
+      }
+    } finally {
+      try {
+        in.close();
+      } catch (IOException exception) {
+        LOG.warn("Problem closing wal: " + exception.getMessage());
+        LOG.debug("exception details.", exception);
+      }
+    }
+    return count;
+  }
+
+  private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master) throws Exception {
+    TEST_UTIL.waitUntilNoRegionsInTransition(60000);
+  }
+
+  private void putData(Region region, byte[] startRow, int numRows, byte [] qf,
+      byte [] ...families)
+      throws IOException {
+    for(int i = 0; i < numRows; i++) {
+      Put put = new Put(Bytes.add(startRow, Bytes.toBytes(i)));
+      for(byte [] family : families) {
+        put.addColumn(family, qf, null);
+      }
+      region.put(put);
+    }
+  }
+
+  /**
+   * Load table with puts and deletes with expected values so that we can verify later
+   */
+  private void prepareData(final Table t, final byte[] f, final byte[] column) throws IOException {
+    byte[] k = new byte[3];
+
+    // add puts
+    List<Put> puts = new ArrayList<>();
+    for (byte b1 = 'a'; b1 <= 'z'; b1++) {
+      for (byte b2 = 'a'; b2 <= 'z'; b2++) {
+        for (byte b3 = 'a'; b3 <= 'z'; b3++) {
+          k[0] = b1;
+          k[1] = b2;
+          k[2] = b3;
+          Put put = new Put(k);
+          put.addColumn(f, column, k);
+          puts.add(put);
+        }
+      }
+    }
+    t.put(puts);
+    // add deletes
+    for (byte b3 = 'a'; b3 <= 'z'; b3++) {
+      k[0] = 'a';
+      k[1] = 'a';
+      k[2] = b3;
+      Delete del = new Delete(k);
+      t.delete(del);
+    }
+  }
+
+  private void waitForCounter(AtomicLong ctr, long oldval, long newval,
+      long timems) {
+    long curt = System.currentTimeMillis();
+    long endt = curt + timems;
+    while (curt < endt) {
+      if (ctr.get() == oldval) {
+        Thread.yield();
+        curt = System.currentTimeMillis();
+      } else {
+        assertEquals(newval, ctr.get());
+        return;
+      }
+    }
+    assertTrue(false);
+  }
+
+  private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException {
+    for (MasterThread mt : cluster.getLiveMasterThreads()) {
+      if (mt.getMaster().isActiveMaster()) {
+        mt.getMaster().abort("Aborting for tests", new Exception("Trace info"));
+        mt.join();
+        break;
+      }
+    }
+    LOG.debug("Master is aborted");
+  }
+
+  /**
+   * Find a RS that has regions of a table.
+   * @param hasMetaRegion when true, the returned RS has hbase:meta region as well
+   * @param tableName
+   * @return
+   * @throws Exception
+   */
+  private HRegionServer findRSToKill(boolean hasMetaRegion, String tableName) throws Exception {
+    List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
+    List<HRegionInfo> regions = null;
+    HRegionServer hrs = null;
+
+    for (RegionServerThread rst: rsts) {
+      hrs = rst.getRegionServer();
+      while (rst.isAlive() && !hrs.isOnline()) {
+        Thread.sleep(100);
+      }
+      if (!rst.isAlive()) {
+        continue;
+      }
+      boolean isCarryingMeta = false;
+      boolean foundTableRegion = false;
+      regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+      for (HRegionInfo region : regions) {
+        if (region.isMetaRegion()) {
+          isCarryingMeta = true;
+        }
+        if (tableName == null || region.getTable().getNameAsString().equals(tableName)) {
+          foundTableRegion = true;
+        }
+        if (foundTableRegion && (isCarryingMeta || !hasMetaRegion)) {
+          break;
+        }
+      }
+      if (isCarryingMeta && hasMetaRegion) {
+        // clients ask for a RS with META
+        if (!foundTableRegion) {
+          final HRegionServer destRS = hrs;
+          // the RS doesn't have regions of the specified table so we need move one to this RS
+          List<HRegionInfo> tableRegions =
+              TEST_UTIL.getHBaseAdmin().getTableRegions(TableName.valueOf(tableName));
+          final HRegionInfo hri = tableRegions.get(0);
+          TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
+              Bytes.toBytes(destRS.getServerName().getServerName()));
+          // wait for region move completes
+          final RegionStates regionStates =
+              TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+          TEST_UTIL.waitFor(45000, 200, new Waiter.Predicate<Exception>() {
+            @Override
+            public boolean evaluate() throws Exception {
+              ServerName sn = regionStates.getRegionServerOfRegion(hri);
+              return (sn != null && sn.equals(destRS.getServerName()));
+            }
+          });
+        }
+        return hrs;
+      } else if (hasMetaRegion || isCarryingMeta) {
+        continue;
+      }
+      if (foundTableRegion) break;
+    }
+
+    return hrs;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
new file mode 100644
index 0000000..395eef2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
@@ -0,0 +1,90 @@
+/**
+ * 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.master.balancer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.testclassification.FlakeyTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({FlakeyTests.class, MediumTests.class})
+public class TestStochasticLoadBalancer2 extends BalancerTestBase {
+  private static final Log LOG = LogFactory.getLog(TestStochasticLoadBalancer2.class);
+
+  @Test (timeout = 800000)
+  public void testRegionReplicasOnMidCluster() {
+    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
+    conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+    TestStochasticLoadBalancer.loadBalancer.setConf(conf);
+    int numNodes = 200;
+    int numRegions = 40 * 200;
+    int replication = 3; // 3 replicas per region
+    int numRegionsPerServer = 30; //all regions are mostly balanced
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+
+  @Test (timeout = 800000)
+  public void testRegionReplicasOnLargeCluster() {
+    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
+    conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+    loadBalancer.setConf(conf);
+    int numNodes = 1000;
+    int numRegions = 20 * numNodes; // 20 * replication regions per RS
+    int numRegionsPerServer = 19; // all servers except one
+    int numTables = 100;
+    int replication = 3;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+  }
+
+  @Test (timeout = 800000)
+  public void testRegionReplicasOnMidClusterHighReplication() {
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 4000000L);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
+    conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+    loadBalancer.setConf(conf);
+    int numNodes = 80;
+    int numRegions = 6 * numNodes;
+    int replication = 80; // 80 replicas per region, one for each server
+    int numRegionsPerServer = 5;
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, true);
+  }
+
+  @Test (timeout = 800000)
+  public void testRegionReplicationOnMidClusterReplicationGreaterThanNumNodes() {
+    conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+    conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
+    conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+    conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+    loadBalancer.setConf(conf);
+    int numNodes = 40;
+    int numRegions = 6 * 50;
+    int replication = 50; // 50 replicas per region, more than numNodes
+    int numRegionsPerServer = 6;
+    int numTables = 10;
+    testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, false);
+  }
+}
\ No newline at end of file


[32/37] hbase git commit: HBASE-15205 Do not find the replication scope for every WAL#append() (Ram)

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 567e09d..e9bb468 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
@@ -22,6 +22,8 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
@@ -208,13 +210,17 @@ public class TestWALLockup {
     HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
     final HRegion region = initHRegion(tableName, null, null, dodgyWAL);
     byte [] bytes = Bytes.toBytes(getName());
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    scopes.put(COLUMN_FAMILY_BYTES, 0);
     try {
       // First get something into memstore. Make a Put and then pull the Cell out of it. Will
       // manage append and sync carefully in below to manufacture hang. We keep adding same
       // edit. WAL subsystem doesn't care.
       Put put = new Put(bytes);
       put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
-      WALKey key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(), htd.getTableName());
+      WALKey key = new WALKey(region.getRegionInfo().getEncodedNameAsBytes(), htd.getTableName(),
+          scopes);
       WALEdit edit = new WALEdit();
       CellScanner CellScanner = put.cellScanner();
       assertTrue(CellScanner.advance());
@@ -228,7 +234,7 @@ public class TestWALLockup {
       LOG.info("SET throwing of exception on append");
       dodgyWAL.throwException = true;
       // This append provokes a WAL roll request
-      dodgyWAL.append(htd, region.getRegionInfo(), key, edit, true);
+      dodgyWAL.append(region.getRegionInfo(), key, edit, true);
       boolean exception = false;
       try {
         dodgyWAL.sync();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
index fd6d535..c60b225 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
@@ -28,7 +28,9 @@ import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
+import java.util.NavigableMap;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.UUID;
 
 import org.apache.commons.lang.mutable.MutableBoolean;
@@ -152,12 +154,9 @@ public class TestFSHLog {
     }
   }
 
-  protected void addEdits(WAL log,
-                          HRegionInfo hri,
-                          HTableDescriptor htd,
-                          int times,
-                          MultiVersionConcurrencyControl mvcc)
-      throws IOException {
+  protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd, int times,
+      MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> scopes)
+          throws IOException {
     final byte[] row = Bytes.toBytes("row");
     for (int i = 0; i < times; i++) {
       long timestamp = System.currentTimeMillis();
@@ -165,8 +164,8 @@ public class TestFSHLog {
       cols.add(new KeyValue(row, row, row, timestamp, row));
       WALKey key = new WALKey(hri.getEncodedNameAsBytes(), htd.getTableName(),
           WALKey.NO_SEQUENCE_ID, timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE,
-          HConstants.NO_NONCE, mvcc);
-      log.append(htd, hri, key, cols, true);
+          HConstants.NO_NONCE, mvcc, scopes);
+      log.append(hri, key, cols, true);
     }
     log.sync();
   }
@@ -261,11 +260,21 @@ public class TestFSHLog {
         new HRegionInfo(t2.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
     // add edits and roll the wal
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
+    NavigableMap<byte[], Integer> scopes1 = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : t1.getFamiliesKeys()) {
+      scopes1.put(fam, 0);
+    }
+    NavigableMap<byte[], Integer> scopes2 = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : t2.getFamiliesKeys()) {
+      scopes2.put(fam, 0);
+    }
     try {
-      addEdits(wal, hri1, t1, 2, mvcc);
+      addEdits(wal, hri1, t1, 2, mvcc, scopes1);
       wal.rollWriter();
       // add some more edits and roll the wal. This would reach the log number threshold
-      addEdits(wal, hri1, t1, 2, mvcc);
+      addEdits(wal, hri1, t1, 2, mvcc, scopes1);
       wal.rollWriter();
       // with above rollWriter call, the max logs limit is reached.
       assertTrue(wal.getNumRolledLogFiles() == 2);
@@ -276,7 +285,7 @@ public class TestFSHLog {
       assertEquals(1, regionsToFlush.length);
       assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]);
       // insert edits in second region
-      addEdits(wal, hri2, t2, 2, mvcc);
+      addEdits(wal, hri2, t2, 2, mvcc, scopes2);
       // get the regions to flush, it should still read region1.
       regionsToFlush = wal.findRegionsToForceFlush();
       assertEquals(regionsToFlush.length, 1);
@@ -293,12 +302,12 @@ public class TestFSHLog {
       // no wal should remain now.
       assertEquals(0, wal.getNumRolledLogFiles());
       // add edits both to region 1 and region 2, and roll.
-      addEdits(wal, hri1, t1, 2, mvcc);
-      addEdits(wal, hri2, t2, 2, mvcc);
+      addEdits(wal, hri1, t1, 2, mvcc, scopes1);
+      addEdits(wal, hri2, t2, 2, mvcc, scopes2);
       wal.rollWriter();
       // add edits and roll the writer, to reach the max logs limit.
       assertEquals(1, wal.getNumRolledLogFiles());
-      addEdits(wal, hri1, t1, 2, mvcc);
+      addEdits(wal, hri1, t1, 2, mvcc, scopes1);
       wal.rollWriter();
       // it should return two regions to flush, as the oldest wal file has entries
       // for both regions.
@@ -310,7 +319,7 @@ public class TestFSHLog {
       wal.rollWriter(true);
       assertEquals(0, wal.getNumRolledLogFiles());
       // Add an edit to region1, and roll the wal.
-      addEdits(wal, hri1, t1, 2, mvcc);
+      addEdits(wal, hri1, t1, 2, mvcc, scopes1);
       // tests partial flush: roll on a partial flush, and ensure that wal is not archived.
       wal.startCacheFlush(hri1.getEncodedNameAsBytes(), t1.getFamiliesKeys());
       wal.rollWriter();
@@ -360,6 +369,11 @@ public class TestFSHLog {
     HBaseTestingUtility.closeRegionAndWAL(r);
     final int countPerFamily = 10;
     final MutableBoolean goslow = new MutableBoolean(false);
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     // subclass and doctor a method.
     FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDefaultRootDirPath(),
         testName, conf) {
@@ -403,9 +417,9 @@ public class TestFSHLog {
       for (int i = 0; i < countPerFamily; i++) {
         final HRegionInfo info = region.getRegionInfo();
         final WALKey logkey = new WALKey(info.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), clusterIds, -1, -1, region.getMVCC());
-        wal.append(htd, info, logkey, edits, true);
-      }
+            System.currentTimeMillis(), clusterIds, -1, -1, region.getMVCC(), scopes);
+        wal.append(info, logkey, edits, true);
+        }
       region.flush(true);
       // FlushResult.flushSequenceId is not visible here so go get the current sequence id.
       long currentSequenceId = region.getReadPoint(null);
@@ -439,11 +453,16 @@ public class TestFSHLog {
       syncRunnerIndexField.set(ringBufferEventHandler, Integer.MAX_VALUE - 1);
       HTableDescriptor htd =
           new HTableDescriptor(TableName.valueOf("t1")).addFamily(new HColumnDescriptor("row"));
+      NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+          Bytes.BYTES_COMPARATOR);
+      for(byte[] fam : htd.getFamiliesKeys()) {
+        scopes.put(fam, 0);
+      }
       HRegionInfo hri =
           new HRegionInfo(htd.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
       MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
       for (int i = 0; i < 10; i++) {
-        addEdits(log, hri, htd, 1, mvcc);
+        addEdits(log, hri, htd, 1, mvcc, scopes);
       }
     } finally {
       log.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index 9dccffe..c05e7f0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -19,6 +19,9 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -199,8 +202,13 @@ public class TestLogRollAbort {
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
         HTableDescriptor htd = new HTableDescriptor(tableName);
         htd.addFamily(new HColumnDescriptor("column"));
-        log.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), mvcc), kvs, true);
+        NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+            Bytes.BYTES_COMPARATOR);
+        for(byte[] fam : htd.getFamiliesKeys()) {
+          scopes.put(fam, 0);
+        }
+        log.append(regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
+            System.currentTimeMillis(), mvcc, scopes), kvs, true);
       }
       // Send the data to HDFS datanodes and close the HDFS writer
       log.sync();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 0c68fc1..9ab7b7d 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import static org.junit.Assert.assertFalse;
 
 import java.io.IOException;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -139,8 +141,13 @@ public class TestLogRollingNoCluster {
           edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
           final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
           final HTableDescriptor htd = TEST_UTIL.getMetaTableDescriptor();
-          final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(),
-              TableName.META_TABLE_NAME, now, mvcc), edit, true);
+          NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+              Bytes.BYTES_COMPARATOR);
+          for(byte[] fam : htd.getFamiliesKeys()) {
+            scopes.put(fam, 0);
+          }
+          final long txid = wal.append(hri, new WALKey(hri.getEncodedNameAsBytes(),
+              TableName.META_TABLE_NAME, now, mvcc, scopes), edit, true);
           wal.sync(txid);
         }
         String msg = getName() + " finished";

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index a2c387b..b6bb7a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -98,9 +100,13 @@ public class TestWALActionsListener {
       edit.add(kv);
       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(SOME_BYTES));
       htd.addFamily(new HColumnDescriptor(b));
-
-      final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(),
-          TableName.valueOf(b), 0), edit, true);
+      NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+          Bytes.BYTES_COMPARATOR);
+      for(byte[] fam : htd.getFamiliesKeys()) {
+        scopes.put(fam, 0);
+      }
+      final long txid = wal.append(hri, new WALKey(hri.getEncodedNameAsBytes(),
+          TableName.valueOf(b), 0, scopes), edit, true);
       wal.sync(txid);
       if (i == 10) {
         wal.registerWALActionsListener(laterobserver);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 dbc06ff..3e894d7 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
@@ -37,7 +37,9 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
+import java.util.NavigableMap;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -308,9 +310,14 @@ public class TestWALReplay {
     // Add 1k to each family.
     final int countPerFamily = 1000;
 
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee,
-          wal1, htd, mvcc);
+          wal1, htd, mvcc, scopes);
     }
     wal1.shutdown();
     runWALSplit(this.conf);
@@ -319,7 +326,7 @@ public class TestWALReplay {
     // Add 1k to each family.
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
-          ee, wal2, htd, mvcc);
+          ee, wal2, htd, mvcc, scopes);
     }
     wal2.shutdown();
     runWALSplit(this.conf);
@@ -800,9 +807,14 @@ public class TestWALReplay {
     // Add 1k to each family.
     final int countPerFamily = 1000;
     Set<byte[]> familyNames = new HashSet<byte[]>();
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     for (HColumnDescriptor hcd: htd.getFamilies()) {
       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
-          ee, wal, htd, mvcc);
+          ee, wal, htd, mvcc, scopes);
       familyNames.add(hcd.getName());
     }
 
@@ -815,13 +827,15 @@ public class TestWALReplay {
     long now = ee.currentTime();
     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
       now, rowName));
-    wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc), edit, true);
+    wal.append(hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit,
+        true);
 
     // Delete the c family to verify deletes make it over.
     edit = new WALEdit();
     now = ee.currentTime();
     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now, KeyValue.Type.DeleteFamily));
-    wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc), edit, true);
+    wal.append(hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit,
+        true);
 
     // Sync.
     wal.sync();
@@ -1046,12 +1060,16 @@ public class TestWALReplay {
     deleteDir(basedir);
 
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
     HBaseTestingUtility.closeRegionAndWAL(region);
     final byte[] family = htd.getColumnFamilies()[0].getName();
     final byte[] rowName = tableName.getName();
-    FSWALEntry entry1 = createFSWALEntry(htd, hri, 1L, rowName, family, ee, mvcc, 1);
-    FSWALEntry entry2 = createFSWALEntry(htd, hri, 2L, rowName, family, ee, mvcc, 2);
+    FSWALEntry entry1 = createFSWALEntry(htd, hri, 1L, rowName, family, ee, mvcc, 1, scopes);
+    FSWALEntry entry2 = createFSWALEntry(htd, hri, 2L, rowName, family, ee, mvcc, 2, scopes);
 
     Path largeFile = new Path(logDir, "wal-1");
     Path smallFile = new Path(logDir, "wal-2");
@@ -1154,8 +1172,8 @@ public class TestWALReplay {
   }
 
   private WALKey createWALKey(final TableName tableName, final HRegionInfo hri,
-      final MultiVersionConcurrencyControl mvcc) {
-    return new WALKey(hri.getEncodedNameAsBytes(), tableName, 999, mvcc);
+      final MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> scopes) {
+    return new WALKey(hri.getEncodedNameAsBytes(), tableName, 999, mvcc, scopes);
   }
 
   private WALEdit createWALEdit(final byte[] rowName, final byte[] family, EnvironmentEdge ee,
@@ -1169,19 +1187,20 @@ public class TestWALReplay {
 
   private FSWALEntry createFSWALEntry(HTableDescriptor htd, HRegionInfo hri, long sequence,
       byte[] rowName, byte[] family, EnvironmentEdge ee, MultiVersionConcurrencyControl mvcc,
-      int index) throws IOException {
+      int index, NavigableMap<byte[], Integer> scopes) throws IOException {
     FSWALEntry entry =
-        new FSWALEntry(sequence, createWALKey(htd.getTableName(), hri, mvcc), createWALEdit(
-          rowName, family, ee, index), htd, hri, true);
+        new FSWALEntry(sequence, createWALKey(htd.getTableName(), hri, mvcc, scopes), createWALEdit(
+          rowName, family, ee, index), hri, true);
     entry.stampRegionSequenceId();
     return entry;
   }
 
   private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName,
       final byte[] family, final int count, EnvironmentEdge ee, final WAL wal,
-      final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc) throws IOException {
+      final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc,
+      NavigableMap<byte[], Integer> scopes) throws IOException {
     for (int j = 0; j < count; j++) {
-      wal.append(htd, hri, createWALKey(tableName, hri, mvcc),
+      wal.append(hri, createWALKey(tableName, hri, mvcc, scopes),
         createWALEdit(rowName, family, ee, j), true);
     }
     wal.sync();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
index e52a600..a50bbc5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java
@@ -18,6 +18,9 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -63,6 +66,7 @@ public class TestReplicationBase {
 
   protected static Table htable1;
   protected static Table htable2;
+  protected static NavigableMap<byte[], Integer> scopes;
 
   protected static HBaseTestingUtility utility1;
   protected static HBaseTestingUtility utility2;
@@ -140,6 +144,11 @@ public class TestReplicationBase {
     table.addFamily(fam);
     fam = new HColumnDescriptor(noRepfamName);
     table.addFamily(fam);
+    scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(HColumnDescriptor f : table.getColumnFamilies()) {
+      scopes.put(f.getName(), f.getScope());
+    }
     Connection connection1 = ConnectionFactory.createConnection(conf1);
     Connection connection2 = ConnectionFactory.createConnection(conf2);
     try (Admin admin1 = connection1.getAdmin()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index ab97238..97ccd33 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -26,6 +26,8 @@ import static org.junit.Assert.fail;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -658,7 +660,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     HRegionInfo hri = new HRegionInfo(htable1.getName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
     WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
-    Replication.scopeWALEdits(htable1.getTableDescriptor(), new WALKey(), edit,
+    Replication.scopeWALEdits(new WALKey(), edit,
       htable1.getConfiguration(), null);
   }
 
@@ -767,7 +769,10 @@ public class TestReplicationSmallTests extends TestReplicationBase {
 
     HRegion region = utility1.getMiniHBaseCluster().getRegions(tableName).get(0);
     HRegionInfo hri = region.getRegionInfo();
-
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : htable1.getTableDescriptor().getFamiliesKeys()) {
+      scopes.put(fam, 1);
+    }
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     int index = utility1.getMiniHBaseCluster().getServerWith(hri.getRegionName());
     WAL wal = utility1.getMiniHBaseCluster().getRegionServer(index).getWAL(region.getRegionInfo());
@@ -778,8 +783,8 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     long now = EnvironmentEdgeManager.currentTime();
     edit.add(new KeyValue(rowName, famName, qualifier,
       now, value));
-    WALKey walKey = new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc);
-    wal.append(htable1.getTableDescriptor(), hri, walKey, edit, true);
+    WALKey walKey = new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes);
+    wal.append(hri, walKey, edit, true);
     wal.sync();
 
     Get get = new Get(rowName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
index 22c421d..c906d6a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
@@ -58,19 +58,19 @@ public class TestReplicationWALEntryFilters {
 
     // meta
     WALKey key1 = new WALKey( HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
-      TableName.META_TABLE_NAME);
+      TableName.META_TABLE_NAME, null);
     Entry metaEntry = new Entry(key1, null);
 
     assertNull(filter.filter(metaEntry));
 
     // ns table
-    WALKey key2 = new WALKey(new byte[] {}, TableName.NAMESPACE_TABLE_NAME);
+    WALKey key2 = new WALKey(new byte[] {}, TableName.NAMESPACE_TABLE_NAME, null);
     Entry nsEntry = new Entry(key2, null);
     assertNull(filter.filter(nsEntry));
 
     // user table
 
-    WALKey key3 = new WALKey(new byte[] {}, TableName.valueOf("foo"));
+    WALKey key3 = new WALKey(new byte[] {}, TableName.valueOf("foo"), null);
     Entry userEntry = new Entry(key3, null);
 
     assertEquals(userEntry, filter.filter(userEntry));
@@ -80,33 +80,30 @@ public class TestReplicationWALEntryFilters {
   public void testScopeWALEntryFilter() {
     ScopeWALEntryFilter filter = new ScopeWALEntryFilter();
 
-    Entry userEntry = createEntry(a, b);
-    Entry userEntryA = createEntry(a);
-    Entry userEntryB = createEntry(b);
-    Entry userEntryEmpty = createEntry();
+    Entry userEntry = createEntry(null, a, b);
+    Entry userEntryA = createEntry(null, a);
+    Entry userEntryB = createEntry(null, b);
+    Entry userEntryEmpty = createEntry(null);
 
     // no scopes
     assertEquals(null, filter.filter(userEntry));
 
     // empty scopes
     TreeMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
-    userEntry = createEntry(a, b);
-    userEntry.getKey().setScopes(scopes);
+    userEntry = createEntry(scopes, a, b);
     assertEquals(null, filter.filter(userEntry));
 
     // different scope
     scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
     scopes.put(c, HConstants.REPLICATION_SCOPE_GLOBAL);
-    userEntry = createEntry(a, b);
-    userEntry.getKey().setScopes(scopes);
+    userEntry = createEntry(scopes, a, b);
     // all kvs should be filtered
     assertEquals(userEntryEmpty, filter.filter(userEntry));
 
     // local scope
     scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
     scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL);
-    userEntry = createEntry(a, b);
-    userEntry.getKey().setScopes(scopes);
+    userEntry = createEntry(scopes, a, b);
     assertEquals(userEntryEmpty, filter.filter(userEntry));
     scopes.put(b, HConstants.REPLICATION_SCOPE_LOCAL);
     assertEquals(userEntryEmpty, filter.filter(userEntry));
@@ -114,8 +111,7 @@ public class TestReplicationWALEntryFilters {
     // only scope a
     scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
     scopes.put(a, HConstants.REPLICATION_SCOPE_GLOBAL);
-    userEntry = createEntry(a, b);
-    userEntry.getKey().setScopes(scopes);
+    userEntry = createEntry(scopes, a, b);
     assertEquals(userEntryA, filter.filter(userEntry));
     scopes.put(b, HConstants.REPLICATION_SCOPE_LOCAL);
     assertEquals(userEntryA, filter.filter(userEntry));
@@ -123,8 +119,7 @@ public class TestReplicationWALEntryFilters {
     // only scope b
     scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
     scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL);
-    userEntry = createEntry(a, b);
-    userEntry.getKey().setScopes(scopes);
+    userEntry = createEntry(scopes, a, b);
     assertEquals(userEntryB, filter.filter(userEntry));
     scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL);
     assertEquals(userEntryB, filter.filter(userEntry));
@@ -132,8 +127,7 @@ public class TestReplicationWALEntryFilters {
     // scope a and b
     scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
     scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL);
-    userEntry = createEntry(a, b);
-    userEntry.getKey().setScopes(scopes);
+    userEntry = createEntry(scopes, a, b);
     assertEquals(userEntryB, filter.filter(userEntry));
     scopes.put(a, HConstants.REPLICATION_SCOPE_LOCAL);
     assertEquals(userEntryB, filter.filter(userEntry));
@@ -155,16 +149,16 @@ public class TestReplicationWALEntryFilters {
 
   @Test
   public void testChainWALEntryFilter() {
-    Entry userEntry = createEntry(a, b, c);
+    Entry userEntry = createEntry(null, a, b, c);
 
     ChainWALEntryFilter filter = new ChainWALEntryFilter(passFilter);
-    assertEquals(createEntry(a,b,c), filter.filter(userEntry));
+    assertEquals(createEntry(null, a,b,c), filter.filter(userEntry));
 
     filter = new ChainWALEntryFilter(passFilter, passFilter);
-    assertEquals(createEntry(a,b,c), filter.filter(userEntry));
+    assertEquals(createEntry(null, a,b,c), filter.filter(userEntry));
 
     filter = new ChainWALEntryFilter(passFilter, passFilter, passFilter);
-    assertEquals(createEntry(a,b,c), filter.filter(userEntry));
+    assertEquals(createEntry(null, a,b,c), filter.filter(userEntry));
 
     filter = new ChainWALEntryFilter(nullFilter);
     assertEquals(null, filter.filter(userEntry));
@@ -189,7 +183,7 @@ public class TestReplicationWALEntryFilters {
           new ChainWALEntryFilter(passFilter),
           new ChainWALEntryFilter(passFilter)),
           new ChainWALEntryFilter(passFilter));
-    assertEquals(createEntry(a,b,c), filter.filter(userEntry));
+    assertEquals(createEntry(null, a,b,c), filter.filter(userEntry));
 
 
     filter =
@@ -206,19 +200,19 @@ public class TestReplicationWALEntryFilters {
     ReplicationPeer peer = mock(ReplicationPeer.class);
 
     when(peer.getTableCFs()).thenReturn(null);
-    Entry userEntry = createEntry(a, b, c);
+    Entry userEntry = createEntry(null, a, b, c);
     TableCfWALEntryFilter filter = new TableCfWALEntryFilter(peer);
-    assertEquals(createEntry(a,b,c), filter.filter(userEntry));
+    assertEquals(createEntry(null, a,b,c), filter.filter(userEntry));
 
     // empty map
-    userEntry = createEntry(a, b, c);
+    userEntry = createEntry(null, a, b, c);
     Map<TableName, List<String>> tableCfs = new HashMap<TableName, List<String>>();
     when(peer.getTableCFs()).thenReturn(tableCfs);
     filter = new TableCfWALEntryFilter(peer);
     assertEquals(null, filter.filter(userEntry));
 
     // table bar
-    userEntry = createEntry(a, b, c);
+    userEntry = createEntry(null, a, b, c);
     tableCfs = new HashMap<TableName, List<String>>();
     tableCfs.put(TableName.valueOf("bar"), null);
     when(peer.getTableCFs()).thenReturn(tableCfs);
@@ -226,24 +220,24 @@ public class TestReplicationWALEntryFilters {
     assertEquals(null, filter.filter(userEntry));
 
     // table foo:a
-    userEntry = createEntry(a, b, c);
+    userEntry = createEntry(null, a, b, c);
     tableCfs = new HashMap<TableName, List<String>>();
     tableCfs.put(TableName.valueOf("foo"), Lists.newArrayList("a"));
     when(peer.getTableCFs()).thenReturn(tableCfs);
     filter = new TableCfWALEntryFilter(peer);
-    assertEquals(createEntry(a), filter.filter(userEntry));
+    assertEquals(createEntry(null, a), filter.filter(userEntry));
 
     // table foo:a,c
-    userEntry = createEntry(a, b, c, d);
+    userEntry = createEntry(null, a, b, c, d);
     tableCfs = new HashMap<TableName, List<String>>();
     tableCfs.put(TableName.valueOf("foo"), Lists.newArrayList("a", "c"));
     when(peer.getTableCFs()).thenReturn(tableCfs);
     filter = new TableCfWALEntryFilter(peer);
-    assertEquals(createEntry(a,c), filter.filter(userEntry));
+    assertEquals(createEntry(null, a,c), filter.filter(userEntry));
   }
 
-  private Entry createEntry(byte[]... kvs) {
-    WALKey key1 = new WALKey(new byte[] {}, TableName.valueOf("foo"));
+  private Entry createEntry(TreeMap<byte[], Integer> scopes, byte[]... kvs) {
+    WALKey key1 = new WALKey(new byte[] {}, TableName.valueOf("foo"), scopes);
     WALEdit edit1 = new WALEdit();
 
     for (byte[] kv : kvs) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index f042a8d..fb8cfa0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -32,6 +32,7 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.SortedMap;
 import java.util.SortedSet;
+import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
@@ -131,6 +132,7 @@ public class TestReplicationSourceManager {
   private static CountDownLatch latch;
 
   private static List<String> files = new ArrayList<String>();
+  private static NavigableMap<byte[], Integer> scopes;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -177,6 +179,11 @@ public class TestReplicationSourceManager {
     col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
     htd.addFamily(col);
 
+    scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     hri = new HRegionInfo(htd.getTableName(), r1, r2);
   }
 
@@ -214,15 +221,20 @@ public class TestReplicationSourceManager {
     manager.init();
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame"));
     htd.addFamily(new HColumnDescriptor(f1));
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     // Testing normal log rolling every 20
     for(long i = 1; i < 101; i++) {
       if(i > 1 && i % 20 == 0) {
         wal.rollWriter();
       }
       LOG.info(i);
-      final long txid = wal.append(htd,
+      final long txid = wal.append(
           hri,
-          new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc),
+          new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
           edit,
           true);
       wal.sync(txid);
@@ -236,8 +248,8 @@ public class TestReplicationSourceManager {
     LOG.info(baseline + " and " + time);
 
     for (int i = 0; i < 3; i++) {
-      wal.append(htd, hri,
-          new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc),
+      wal.append(hri,
+          new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
           edit,
           true);
     }
@@ -254,8 +266,8 @@ public class TestReplicationSourceManager {
     manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
         "1", 0, false, false);
 
-    wal.append(htd, hri,
-        new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc),
+    wal.append(hri,
+        new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
         edit,
         true);
     wal.sync();
@@ -427,33 +439,35 @@ public class TestReplicationSourceManager {
 
   @Test
   public void testBulkLoadWALEditsWithoutBulkLoadReplicationEnabled() throws Exception {
-    // 1. Create wal key
-    WALKey logKey = new WALKey();
-    // 2. Get the bulk load wal edit event
-    WALEdit logEdit = getBulkLoadWALEdit();
+    NavigableMap<byte[], Integer> scope = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    // 1. Get the bulk load wal edit event
+    WALEdit logEdit = getBulkLoadWALEdit(scope);
+    // 2. Create wal key
+    WALKey logKey = new WALKey(scope);
 
     // 3. Get the scopes for the key
-    Replication.scopeWALEdits(htd, logKey, logEdit, conf, manager);
+    Replication.scopeWALEdits(logKey, logEdit, conf, manager);
 
     // 4. Assert that no bulk load entry scopes are added if bulk load hfile replication is disabled
-    assertNull("No bulk load entries scope should be added if bulk load replication is diabled.",
-      logKey.getScopes());
+    assertNull("No bulk load entries scope should be added if bulk load replication is disabled.",
+      logKey.getReplicationScopes());
   }
 
   @Test
   public void testBulkLoadWALEdits() throws Exception {
-    // 1. Create wal key
-    WALKey logKey = new WALKey();
-    // 2. Get the bulk load wal edit event
-    WALEdit logEdit = getBulkLoadWALEdit();
+    // 1. Get the bulk load wal edit event
+    NavigableMap<byte[], Integer> scope = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    WALEdit logEdit = getBulkLoadWALEdit(scope);
+    // 2. Create wal key
+    WALKey logKey = new WALKey(scope);
     // 3. Enable bulk load hfile replication
     Configuration bulkLoadConf = HBaseConfiguration.create(conf);
     bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
 
     // 4. Get the scopes for the key
-    Replication.scopeWALEdits(htd, logKey, logEdit, bulkLoadConf, manager);
+    Replication.scopeWALEdits(logKey, logEdit, bulkLoadConf, manager);
 
-    NavigableMap<byte[], Integer> scopes = logKey.getScopes();
+    NavigableMap<byte[], Integer> scopes = logKey.getReplicationScopes();
     // Assert family with replication scope global is present in the key scopes
     assertTrue("This family scope is set to global, should be part of replication key scopes.",
       scopes.containsKey(f1));
@@ -462,17 +476,16 @@ public class TestReplicationSourceManager {
       scopes.containsKey(f2));
   }
 
-  private WALEdit getBulkLoadWALEdit() {
+  private WALEdit getBulkLoadWALEdit(NavigableMap<byte[], Integer> scope) {
     // 1. Create store files for the families
     Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
     List<Path> p = new ArrayList<>(1);
     p.add(new Path(Bytes.toString(f1)));
     storeFiles.put(f1, p);
-
+    scope.put(f1, 1);
     p = new ArrayList<>(1);
     p.add(new Path(Bytes.toString(f2)));
     storeFiles.put(f2, p);
-
     // 2. Create bulk load descriptor
     BulkLoadDescriptor desc = ProtobufUtil.toBulkLoadDescriptor(hri.getTable(),
       ByteStringer.wrap(hri.getEncodedNameAsBytes()), storeFiles, 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java
index 2ad34ea..3ef658f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationWALReaderManager.java
@@ -28,6 +28,8 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.conf.Configuration;
@@ -75,6 +77,7 @@ public class TestReplicationWALReaderManager {
   private static final HRegionInfo info = new HRegionInfo(tableName,
       HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false);
   private static final HTableDescriptor htd = new HTableDescriptor(tableName);
+  private static NavigableMap<byte[], Integer> scopes;
 
   private WAL log;
   private ReplicationWALReaderManager logManager;
@@ -123,6 +126,11 @@ public class TestReplicationWALReaderManager {
 
     cluster = TEST_UTIL.getDFSCluster();
     fs = cluster.getFileSystem();
+    scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
   }
 
   @AfterClass
@@ -204,9 +212,8 @@ public class TestReplicationWALReaderManager {
   }
 
   private void appendToLogPlus(int count) throws IOException {
-    final long txid = log.append(htd, info,
-        new WALKey(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc),
-        getWALEdits(count), true);
+    final long txid = log.append(info, new WALKey(info.getEncodedNameAsBytes(), tableName,
+        System.currentTimeMillis(), mvcc, scopes), getWALEdits(count), true);
     log.sync(txid);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 6eac388..79b94cf 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
@@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 
 // imports for things that haven't moved yet
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
@@ -60,12 +59,12 @@ public class FaultyFSLog extends FSHLog {
   }
 
   @Override
-  public long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits,
-      boolean inMemstore) throws IOException {
+  public long append(HRegionInfo info, WALKey key,
+      WALEdit edits, boolean inMemstore) throws IOException {
     if (this.ft == FailureType.APPEND) {
       throw new IOException("append");
     }
-    return super.append(htd, info, key, edits, inMemstore);
+    return super.append(info, key, edits, inMemstore);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
index 89c63a6..9b6ac54 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProvider.java
@@ -25,8 +25,10 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.NavigableMap;
 import java.util.Random;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
@@ -151,23 +153,25 @@ public class TestDefaultWALProvider {
 
 
   protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd,
-                        int times) throws IOException {
+                        int times, NavigableMap<byte[], Integer> scopes) throws IOException {
     final byte[] row = Bytes.toBytes("row");
     for (int i = 0; i < times; i++) {
       long timestamp = System.currentTimeMillis();
       WALEdit cols = new WALEdit();
       cols.add(new KeyValue(row, row, row, timestamp, row));
-      log.append(htd, hri, getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp),
-        cols, true);
+      log.append(hri, getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp, scopes),
+          cols, true);
     }
     log.sync();
   }
 
   /**
    * used by TestDefaultWALProviderWithHLogKey
+   * @param scopes
    */
-  WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp) {
-    return new WALKey(info, tableName, timestamp, mvcc);
+  WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp,
+      NavigableMap<byte[], Integer> scopes) {
+    return new WALKey(info, tableName, timestamp, mvcc, scopes);
   }
 
   /**
@@ -191,6 +195,16 @@ public class TestDefaultWALProvider {
     final HTableDescriptor htd2 =
         new HTableDescriptor(TableName.valueOf("testLogCleaning2"))
             .addFamily(new HColumnDescriptor("row"));
+    NavigableMap<byte[], Integer> scopes1 = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes1.put(fam, 0);
+    }
+    NavigableMap<byte[], Integer> scopes2 = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd2.getFamiliesKeys()) {
+      scopes2.put(fam, 0);
+    }
     final Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, DefaultWALProvider.class.getName());
     final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
@@ -205,26 +219,26 @@ public class TestDefaultWALProvider {
 
       // Add a single edit and make sure that rolling won't remove the file
       // Before HBASE-3198 it used to delete it
-      addEdits(log, hri, htd, 1);
+      addEdits(log, hri, htd, 1, scopes1);
       log.rollWriter();
       assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(log));
 
       // See if there's anything wrong with more than 1 edit
-      addEdits(log, hri, htd, 2);
+      addEdits(log, hri, htd, 2, scopes1);
       log.rollWriter();
       assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(log));
 
       // Now mix edits from 2 regions, still no flushing
-      addEdits(log, hri, htd, 1);
-      addEdits(log, hri2, htd2, 1);
-      addEdits(log, hri, htd, 1);
-      addEdits(log, hri2, htd2, 1);
+      addEdits(log, hri, htd, 1, scopes1);
+      addEdits(log, hri2, htd2, 1, scopes2);
+      addEdits(log, hri, htd, 1, scopes1);
+      addEdits(log, hri2, htd2, 1, scopes2);
       log.rollWriter();
       assertEquals(3, DefaultWALProvider.getNumRolledLogFiles(log));
 
       // Flush the first region, we expect to see the first two files getting
       // archived. We need to append something or writer won't be rolled.
-      addEdits(log, hri2, htd2, 1);
+      addEdits(log, hri2, htd2, 1, scopes2);
       log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys());
       log.completeCacheFlush(hri.getEncodedNameAsBytes());
       log.rollWriter();
@@ -233,7 +247,7 @@ public class TestDefaultWALProvider {
       // Flush the second region, which removes all the remaining output files
       // since the oldest was completely flushed and the two others only contain
       // flush information
-      addEdits(log, hri2, htd2, 1);
+      addEdits(log, hri2, htd2, 1, scopes2);
       log.startCacheFlush(hri2.getEncodedNameAsBytes(), htd2.getFamiliesKeys());
       log.completeCacheFlush(hri2.getEncodedNameAsBytes());
       log.rollWriter();
@@ -264,6 +278,16 @@ public class TestDefaultWALProvider {
         new HTableDescriptor(TableName.valueOf("t1")).addFamily(new HColumnDescriptor("row"));
     HTableDescriptor table2 =
         new HTableDescriptor(TableName.valueOf("t2")).addFamily(new HColumnDescriptor("row"));
+    NavigableMap<byte[], Integer> scopes1 = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : table1.getFamiliesKeys()) {
+      scopes1.put(fam, 0);
+    }
+    NavigableMap<byte[], Integer> scopes2 = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : table2.getFamiliesKeys()) {
+      scopes2.put(fam, 0);
+    }
     final Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, DefaultWALProvider.class.getName());
     final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
@@ -281,31 +305,31 @@ public class TestDefaultWALProvider {
       hri2.setSplit(false);
       // variables to mock region sequenceIds.
       // start with the testing logic: insert a waledit, and roll writer
-      addEdits(wal, hri1, table1, 1);
+      addEdits(wal, hri1, table1, 1, scopes1);
       wal.rollWriter();
       // assert that the wal is rolled
       assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(wal));
       // add edits in the second wal file, and roll writer.
-      addEdits(wal, hri1, table1, 1);
+      addEdits(wal, hri1, table1, 1, scopes1);
       wal.rollWriter();
       // assert that the wal is rolled
       assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal));
       // add a waledit to table1, and flush the region.
-      addEdits(wal, hri1, table1, 3);
+      addEdits(wal, hri1, table1, 3, scopes1);
       flushRegion(wal, hri1.getEncodedNameAsBytes(), table1.getFamiliesKeys());
       // roll log; all old logs should be archived.
       wal.rollWriter();
       assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(wal));
       // add an edit to table2, and roll writer
-      addEdits(wal, hri2, table2, 1);
+      addEdits(wal, hri2, table2, 1, scopes2);
       wal.rollWriter();
       assertEquals(1, DefaultWALProvider.getNumRolledLogFiles(wal));
       // add edits for table1, and roll writer
-      addEdits(wal, hri1, table1, 2);
+      addEdits(wal, hri1, table1, 2, scopes1);
       wal.rollWriter();
       assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal));
       // add edits for table2, and flush hri1.
-      addEdits(wal, hri2, table2, 2);
+      addEdits(wal, hri2, table2, 2, scopes2);
       flushRegion(wal, hri1.getEncodedNameAsBytes(), table2.getFamiliesKeys());
       // the log : region-sequenceId map is
       // log1: region2 (unflushed)
@@ -315,7 +339,7 @@ public class TestDefaultWALProvider {
       wal.rollWriter();
       assertEquals(2, DefaultWALProvider.getNumRolledLogFiles(wal));
       // flush region2, and all logs should be archived.
-      addEdits(wal, hri2, table2, 2);
+      addEdits(wal, hri2, table2, 2, scopes2);
       flushRegion(wal, hri2.getEncodedNameAsBytes(), table2.getFamiliesKeys());
       wal.rollWriter();
       assertEquals(0, DefaultWALProvider.getNumRolledLogFiles(wal));

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java
index 1885d87..ef92768 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestDefaultWALProviderWithHLogKey.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hbase.wal;
 
 
+import java.util.NavigableMap;
+
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -28,7 +30,8 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 @Category({RegionServerTests.class, LargeTests.class})
 public class TestDefaultWALProviderWithHLogKey extends TestDefaultWALProvider {
   @Override
-  WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp) {
-    return new HLogKey(info, tableName, timestamp, mvcc);
+  WALKey getWalKey(final byte[] info, final TableName tableName, final long timestamp,
+      final NavigableMap<byte[], Integer> scopes) {
+    return new HLogKey(info, tableName, timestamp, mvcc, scopes);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
index 079e0cb..caa0a45 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
@@ -79,6 +81,11 @@ public class TestSecureWAL {
     TableName tableName = TableName.valueOf("TestSecureWAL");
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor(tableName.getName()));
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     HRegionInfo regioninfo = new HRegionInfo(tableName,
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
     final int total = 10;
@@ -95,8 +102,8 @@ public class TestSecureWAL {
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
-      wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
-          System.currentTimeMillis()), kvs, true);
+      wal.append(regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
+          System.currentTimeMillis(), scopes), kvs, true);
     }
     wal.sync();
     final Path walPath = DefaultWALProvider.getCurrentFileName(wal);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 747977a..0eef3b1 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
@@ -30,6 +30,8 @@ import java.io.IOException;
 import java.lang.reflect.Method;
 import java.net.BindException;
 import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -181,6 +183,11 @@ public class TestWALFactory {
     }
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor("column"));
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
 
     // Add edits for three regions.
     for (int ii = 0; ii < howmany; ii++) {
@@ -196,8 +203,8 @@ public class TestWALFactory {
               System.currentTimeMillis(), column));
           LOG.info("Region " + i + ": " + edit);
           WALKey walKey =  new WALKey(infos[i].getEncodedNameAsBytes(), tableName,
-              System.currentTimeMillis(), mvcc);
-          log.append(htd, infos[i], walKey, edit, true);
+              System.currentTimeMillis(), mvcc, scopes);
+          log.append(infos[i], walKey, edit, true);
           walKey.getWriteEntry();
         }
         log.sync();
@@ -249,13 +256,18 @@ public class TestWALFactory {
                   null,null, false);
       HTableDescriptor htd = new HTableDescriptor(tableName);
       htd.addFamily(new HColumnDescriptor(tableName.getName()));
+      NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+          Bytes.BYTES_COMPARATOR);
+      for(byte[] fam : htd.getFamiliesKeys()) {
+        scopes.put(fam, 0);
+      }
       final WAL wal = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
 
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-        wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), mvcc), kvs, true);
+        wal.append(info, new WALKey(info.getEncodedNameAsBytes(), tableName,
+            System.currentTimeMillis(), mvcc, scopes), kvs, true);
       }
       // Now call sync and try reading.  Opening a Reader before you sync just
       // gives you EOFE.
@@ -273,8 +285,8 @@ public class TestWALFactory {
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-        wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), mvcc), kvs, true);
+        wal.append(info, new WALKey(info.getEncodedNameAsBytes(), tableName,
+            System.currentTimeMillis(), mvcc, scopes), kvs, true);
       }
       wal.sync();
       reader = wals.createReader(fs, walPath);
@@ -295,8 +307,8 @@ public class TestWALFactory {
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), value));
-        wal.append(htd, info, new WALKey(info.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), mvcc), kvs,  true);
+        wal.append(info, new WALKey(info.getEncodedNameAsBytes(), tableName,
+            System.currentTimeMillis(), mvcc, scopes), kvs,  true);
       }
       // Now I should have written out lots of blocks.  Sync then read.
       wal.sync();
@@ -370,12 +382,17 @@ public class TestWALFactory {
 
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor(tableName.getName()));
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
 
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-      wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
-          System.currentTimeMillis()), kvs,  true);
+      wal.append(regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
+          System.currentTimeMillis(), scopes), kvs,  true);
     }
     // Now call sync to send the data to HDFS datanodes
     wal.sync();
@@ -485,6 +502,11 @@ public class TestWALFactory {
     final HTableDescriptor htd =
         new HTableDescriptor(TableName.valueOf("tablename")).addFamily(new HColumnDescriptor(
             "column"));
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     final byte [] row = Bytes.toBytes("row");
     WAL.Reader reader = null;
     try {
@@ -503,9 +525,9 @@ public class TestWALFactory {
         row,Bytes.toBytes(Bytes.toString(row) + "1"), false);
       final WAL log = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
 
-      final long txid = log.append(htd, info,
+      final long txid = log.append(info,
         new WALKey(info.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(),
-            mvcc),
+            mvcc, scopes),
         cols, true);
       log.sync(txid);
       log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getFamiliesKeys());
@@ -545,6 +567,11 @@ public class TestWALFactory {
     final HTableDescriptor htd =
         new HTableDescriptor(TableName.valueOf("tablename")).addFamily(new HColumnDescriptor(
             "column"));
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     final byte [] row = Bytes.toBytes("row");
     WAL.Reader reader = null;
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
@@ -561,9 +588,9 @@ public class TestWALFactory {
       HRegionInfo hri = new HRegionInfo(htd.getTableName(),
           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
       final WAL log = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
-      final long txid = log.append(htd, hri,
+      final long txid = log.append(hri,
         new WALKey(hri.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(),
-            mvcc),
+            mvcc, scopes),
         cols, true);
       log.sync(txid);
       log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys());
@@ -607,7 +634,11 @@ public class TestWALFactory {
     long timestamp = System.currentTimeMillis();
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor("column"));
-
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     HRegionInfo hri = new HRegionInfo(tableName,
         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
     final WAL log = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
@@ -617,8 +648,8 @@ public class TestWALFactory {
       cols.add(new KeyValue(row, Bytes.toBytes("column"),
           Bytes.toBytes(Integer.toString(i)),
           timestamp, new byte[]{(byte) (i + '0')}));
-      log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
-          System.currentTimeMillis(), mvcc), cols, true);
+      log.append(hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
+          System.currentTimeMillis(), mvcc, scopes), cols, true);
     }
     log.sync();
     assertEquals(COL_COUNT, visitor.increments);
@@ -627,8 +658,8 @@ public class TestWALFactory {
     cols.add(new KeyValue(row, Bytes.toBytes("column"),
         Bytes.toBytes(Integer.toString(11)),
         timestamp, new byte[]{(byte) (11 + '0')}));
-    log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
-        System.currentTimeMillis(), mvcc), cols, true);
+    log.append(hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
+        System.currentTimeMillis(), mvcc, scopes), cols, true);
     log.sync();
     assertEquals(COL_COUNT, visitor.increments);
   }
@@ -722,8 +753,9 @@ public class TestWALFactory {
     }
 
     @Override
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {
-      //To change body of implemented methods use File | Settings | File Templates.
+    public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
+      // To change body of implemented methods use File | Settings | File
+      // Templates.
       increments++;
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
index 9ae98c6..beac9e2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
@@ -21,6 +21,8 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.LogFactory;
@@ -96,6 +98,11 @@ public class TestWALReaderOnSecureWAL {
       TableName tableName = TableName.valueOf(tblName);
       HTableDescriptor htd = new HTableDescriptor(tableName);
       htd.addFamily(new HColumnDescriptor(tableName.getName()));
+      NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+          Bytes.BYTES_COMPARATOR);
+      for(byte[] fam : htd.getFamiliesKeys()) {
+        scopes.put(fam, 0);
+      }
       HRegionInfo regioninfo = new HRegionInfo(tableName,
         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
       final int total = 10;
@@ -109,8 +116,8 @@ public class TestWALReaderOnSecureWAL {
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
-        wal.append(htd, regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), mvcc), kvs, true);
+        wal.append(regioninfo, new WALKey(regioninfo.getEncodedNameAsBytes(), tableName,
+            System.currentTimeMillis(), mvcc, scopes), kvs, true);
       }
       wal.sync();
       final Path walPath = DefaultWALProvider.getCurrentFileName(wal);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index e138174..4a15d3c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -23,8 +23,10 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.Random;
 import java.util.Set;
+import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
@@ -128,6 +130,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
     private final int syncInterval;
     private final HTableDescriptor htd;
     private final Sampler loopSampler;
+    private final NavigableMap<byte[], Integer> scopes;
 
     WALPutBenchmark(final HRegion region, final HTableDescriptor htd,
         final long numIterations, final boolean noSync, final int syncInterval,
@@ -138,6 +141,11 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
       this.numFamilies = htd.getColumnFamilies().length;
       this.region = region;
       this.htd = htd;
+      scopes = new TreeMap<byte[], Integer>(
+          Bytes.BYTES_COMPARATOR);
+      for(byte[] fam : htd.getFamiliesKeys()) {
+        scopes.put(fam, 0);
+      }
       String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes");
       if (spanReceivers == null || spanReceivers.isEmpty()) {
         loopSampler = Sampler.NEVER;
@@ -180,8 +188,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
             addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit);
             HRegionInfo hri = region.getRegionInfo();
             final WALKey logkey =
-                new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc);
-            wal.append(htd, hri, logkey, walEdit, true);
+                new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc, scopes);
+            wal.append(hri, logkey, walEdit, true);
             if (!this.noSync) {
               if (++lastSync >= this.syncInterval) {
                 wal.sync();
@@ -498,8 +506,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
         private int appends = 0;
 
         @Override
-        public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey,
-            WALEdit logEdit) {
+        public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
           this.appends++;
           if (this.appends % whenToRoll == 0) {
             LOG.info("Rolling after " + appends + " edits");


[23/37] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
index 4371739..0240a67 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
@@ -8196,6 +8196,450 @@ public final class ZooKeeperProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.TableLock)
   }
 
+  public interface SwitchStateOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional bool enabled = 1;
+    /**
+     * <code>optional bool enabled = 1;</code>
+     */
+    boolean hasEnabled();
+    /**
+     * <code>optional bool enabled = 1;</code>
+     */
+    boolean getEnabled();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.SwitchState}
+   *
+   * <pre>
+   **
+   * State of the switch.
+   * </pre>
+   */
+  public static final class SwitchState extends
+      com.google.protobuf.GeneratedMessage
+      implements SwitchStateOrBuilder {
+    // Use SwitchState.newBuilder() to construct.
+    private SwitchState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SwitchState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SwitchState defaultInstance;
+    public static SwitchState getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SwitchState getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SwitchState(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              enabled_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SwitchState> PARSER =
+        new com.google.protobuf.AbstractParser<SwitchState>() {
+      public SwitchState parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SwitchState(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SwitchState> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional bool enabled = 1;
+    public static final int ENABLED_FIELD_NUMBER = 1;
+    private boolean enabled_;
+    /**
+     * <code>optional bool enabled = 1;</code>
+     */
+    public boolean hasEnabled() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional bool enabled = 1;</code>
+     */
+    public boolean getEnabled() {
+      return enabled_;
+    }
+
+    private void initFields() {
+      enabled_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBool(1, enabled_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(1, enabled_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState) obj;
+
+      boolean result = true;
+      result = result && (hasEnabled() == other.hasEnabled());
+      if (hasEnabled()) {
+        result = result && (getEnabled()
+            == other.getEnabled());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasEnabled()) {
+        hash = (37 * hash) + ENABLED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getEnabled());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.SwitchState}
+     *
+     * <pre>
+     **
+     * State of the switch.
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchStateOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        enabled_ = false;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState build() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.enabled_ = enabled_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.getDefaultInstance()) return this;
+        if (other.hasEnabled()) {
+          setEnabled(other.getEnabled());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional bool enabled = 1;
+      private boolean enabled_ ;
+      /**
+       * <code>optional bool enabled = 1;</code>
+       */
+      public boolean hasEnabled() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional bool enabled = 1;</code>
+       */
+      public boolean getEnabled() {
+        return enabled_;
+      }
+      /**
+       * <code>optional bool enabled = 1;</code>
+       */
+      public Builder setEnabled(boolean value) {
+        bitField0_ |= 0x00000001;
+        enabled_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool enabled = 1;</code>
+       */
+      public Builder clearEnabled() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        enabled_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SwitchState)
+    }
+
+    static {
+      defaultInstance = new SwitchState(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.SwitchState)
+  }
+
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MetaRegionServer_descriptor;
   private static
@@ -8246,6 +8690,11 @@ public final class ZooKeeperProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_TableLock_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_SwitchState_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_SwitchState_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -8286,9 +8735,10 @@ public final class ZooKeeperProtos {
       "\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableNam" +
       "e\022(\n\nlock_owner\030\002 \001(\0132\024.hbase.pb.ServerN",
       "ame\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(" +
-      "\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003B" +
-      "E\n*org.apache.hadoop.hbase.protobuf.gene" +
-      "ratedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
+      "\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\"" +
+      "\036\n\013SwitchState\022\017\n\007enabled\030\001 \001(\010BE\n*org.a" +
+      "pache.hadoop.hbase.protobuf.generatedB\017Z" +
+      "ooKeeperProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8355,6 +8805,12 @@ public final class ZooKeeperProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_TableLock_descriptor,
               new java.lang.String[] { "TableName", "LockOwner", "ThreadId", "IsShared", "Purpose", "CreateTime", });
+          internal_static_hbase_pb_SwitchState_descriptor =
+            getDescriptor().getMessageTypes().get(10);
+          internal_static_hbase_pb_SwitchState_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_SwitchState_descriptor,
+              new java.lang.String[] { "Enabled", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index aa31a5e..79bb862 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -279,6 +279,29 @@ message IsBalancerEnabledResponse {
   required bool enabled = 1;
 }
 
+enum MasterSwitchType {
+  SPLIT = 0;
+  MERGE = 1;
+}
+
+message SetSplitOrMergeEnabledRequest {
+  required bool enabled = 1;
+  optional bool synchronous = 2;
+  repeated MasterSwitchType switch_types = 3;
+}
+
+message SetSplitOrMergeEnabledResponse {
+  repeated bool prev_value = 1;
+}
+
+message IsSplitOrMergeEnabledRequest {
+  required MasterSwitchType switch_type = 1;
+}
+
+message IsSplitOrMergeEnabledResponse {
+  required bool enabled = 1;
+}
+
 message NormalizeRequest {
 }
 
@@ -633,6 +656,19 @@ service MasterService {
     returns(IsBalancerEnabledResponse);
 
   /**
+   * Turn the split or merge switch on or off.
+   * If synchronous is true, it waits until current operation call, if outstanding, to return.
+   */
+  rpc SetSplitOrMergeEnabled(SetSplitOrMergeEnabledRequest)
+    returns(SetSplitOrMergeEnabledResponse);
+
+  /**
+   * Query whether the split or merge switch is on/off.
+   */
+  rpc IsSplitOrMergeEnabled(IsSplitOrMergeEnabledRequest)
+    returns(IsSplitOrMergeEnabledResponse);
+
+  /**
    * Run region normalizer. Can NOT run for various reasons. Check logs.
    */
   rpc Normalize(NormalizeRequest)

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-protocol/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
index 54652af..4963c09 100644
--- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
@@ -153,3 +153,10 @@ message TableLock {
   optional string purpose = 5;
   optional int64 create_time = 6;
 }
+
+/**
+ * State of the switch.
+ */
+message SwitchState {
+  optional bool enabled = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 53a080e..1110db3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.RegionStateListener;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
@@ -2353,6 +2354,11 @@ public class AssignmentManager {
       return hri.getShortNameToLog() + " is not opening on " + serverName;
     }
 
+    if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
+            Admin.MasterSwitchType.SPLIT)) {
+      return "split switch is off!";
+    }
+
     // Just return in case of retrying
     if (current.isSplitting()) {
       return null;
@@ -2511,6 +2517,10 @@ public class AssignmentManager {
       return "Merging daughter region already exists, p=" + current;
     }
 
+    if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
+            Admin.MasterSwitchType.MERGE)) {
+      return "merge switch is off!";
+    }
     // Just return in case of retrying
     if (current != null) {
       return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 5d8c325..6806c2d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
@@ -155,6 +156,7 @@ import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
+import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTrackerManager;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -253,6 +255,9 @@ public class HMaster extends HRegionServer implements MasterServices {
   // Tracker for load balancer state
   LoadBalancerTracker loadBalancerTracker;
 
+  // Tracker for split and merge state
+  SplitOrMergeTrackerManager splitOrMergeTracker;
+
   // Tracker for region normalizer state
   private RegionNormalizerTracker regionNormalizerTracker;
 
@@ -578,8 +583,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.normalizer.setMasterServices(this);
     this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
     this.loadBalancerTracker.start();
+
     this.regionNormalizerTracker = new RegionNormalizerTracker(zooKeeper, this);
     this.regionNormalizerTracker.start();
+
+    this.splitOrMergeTracker = new SplitOrMergeTrackerManager(zooKeeper, conf, this);
+    this.splitOrMergeTracker.start();
+
     this.assignmentManager = new AssignmentManager(this, serverManager,
       this.balancer, this.service, this.metricsMaster,
       this.tableLockManager, tableStateManager);
@@ -2783,6 +2793,20 @@ public class HMaster extends HRegionServer implements MasterServices {
     return null == regionNormalizerTracker? false: regionNormalizerTracker.isNormalizerOn();
   }
 
+
+  /**
+   * Queries the state of the {@link SplitOrMergeTrackerManager}. If it is not initialized,
+   * false is returned. If switchType is illegal, false will return.
+   * @param switchType see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}
+   * @return The state of the switch
+   */
+  public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) {
+    if (null == splitOrMergeTracker) {
+      return false;
+    }
+    return splitOrMergeTracker.isSplitOrMergeEnabled(switchType);
+  }
+
   /**
    * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
    *
@@ -2799,4 +2823,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   public RegionNormalizerTracker getRegionNormalizerTracker() {
     return regionNormalizerTracker;
   }
+
+  public SplitOrMergeTrackerManager getSplitOrMergeTracker() {
+    return splitOrMergeTracker;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 1dd4c14..1c770d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
@@ -1506,6 +1507,35 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(RpcController controller,
+    SetSplitOrMergeEnabledRequest request) throws ServiceException {
+    SetSplitOrMergeEnabledResponse.Builder response = SetSplitOrMergeEnabledResponse.newBuilder();
+    try {
+      master.checkInitialized();
+      for (MasterSwitchType masterSwitchType : request.getSwitchTypesList()) {
+        Admin.MasterSwitchType switchType = convert(masterSwitchType);
+        boolean oldValue = master.isSplitOrMergeEnabled(switchType);
+        boolean newValue = request.getEnabled();
+        master.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue, switchType);
+        response.addPrevValue(oldValue);
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    } catch (KeeperException e) {
+      throw new ServiceException(e);
+    }
+    return response.build();
+  }
+
+  @Override
+  public IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(RpcController controller,
+    IsSplitOrMergeEnabledRequest request) throws ServiceException {
+    IsSplitOrMergeEnabledResponse.Builder response = IsSplitOrMergeEnabledResponse.newBuilder();
+    response.setEnabled(master.isSplitOrMergeEnabled(convert(request.getSwitchType())));
+    return response.build();
+  }
+
+  @Override
   public NormalizeResponse normalize(RpcController controller,
       NormalizeRequest request) throws ServiceException {
     try {
@@ -1574,4 +1604,16 @@ public class MasterRpcServices extends RSRpcServices
     }
     return response.build();
   }
+
+  private Admin.MasterSwitchType convert(MasterSwitchType switchType) {
+    switch (switchType) {
+      case SPLIT:
+        return Admin.MasterSwitchType.SPLIT;
+      case MERGE:
+        return Admin.MasterSwitchType.MERGE;
+      default:
+        break;
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index a9113ec..93287ad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -235,6 +235,7 @@ public class HBaseFsck extends Configured implements Closeable {
   private long timelag = DEFAULT_TIME_LAG; // tables whose modtime is older
   private static boolean forceExclusive = false; // only this hbck can modify HBase
   private static boolean disableBalancer = false; // disable load balancer to keep regions stable
+  private static boolean disableSplitAndMerge = false; // disable split and merge
   private boolean fixAssignments = false; // fix assignment errors?
   private boolean fixMeta = false; // fix meta errors?
   private boolean checkHdfs = true; // load and check fs consistency?
@@ -683,6 +684,11 @@ public class HBaseFsck extends Configured implements Closeable {
     if (shouldDisableBalancer()) {
       oldBalancer = admin.setBalancerRunning(false, true);
     }
+    boolean[] oldSplitAndMerge = null;
+    if (shouldDisableSplitAndMerge()) {
+      oldSplitAndMerge = admin.setSplitOrMergeEnabled(false, false,
+        Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
+    }
 
     try {
       onlineConsistencyRepair();
@@ -694,6 +700,17 @@ public class HBaseFsck extends Configured implements Closeable {
       if (shouldDisableBalancer() && oldBalancer) {
         admin.setBalancerRunning(oldBalancer, false);
       }
+
+      if (shouldDisableSplitAndMerge()) {
+        if (oldSplitAndMerge != null) {
+          if (oldSplitAndMerge[0]) {
+            admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
+          }
+          if (oldSplitAndMerge[1]) {
+            admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
+          }
+        }
+      }
     }
 
     if (checkRegionBoundaries) {
@@ -4184,6 +4201,13 @@ public class HBaseFsck extends Configured implements Closeable {
   }
 
   /**
+   * Disable the split and merge
+   */
+  public static void setDisableSplitAndMerge() {
+    disableSplitAndMerge = true;
+  }
+
+  /**
    * The balancer should be disabled if we are modifying HBase.
    * It can be disabled if you want to prevent region movement from causing
    * false positives.
@@ -4193,6 +4217,15 @@ public class HBaseFsck extends Configured implements Closeable {
   }
 
   /**
+   * The split and merge should be disabled if we are modifying HBase.
+   * It can be disabled if you want to prevent region movement from causing
+   * false positives.
+   */
+  public boolean shouldDisableSplitAndMerge() {
+    return fixAny || disableSplitAndMerge;
+  }
+
+  /**
    * Set summary mode.
    * Print only summary of the tables and status (OK or INCONSISTENT)
    */
@@ -4551,6 +4584,8 @@ public class HBaseFsck extends Configured implements Closeable {
         setForceExclusive();
       } else if (cmd.equals("-disableBalancer")) {
         setDisableBalancer();
+      }  else if (cmd.equals("-disableSplitAndMerge")) {
+        setDisableSplitAndMerge();
       } else if (cmd.equals("-timelag")) {
         if (i == args.length - 1) {
           errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -timelag needs a value.");

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java
new file mode 100644
index 0000000..1495dd1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java
@@ -0,0 +1,151 @@
+/**
+ * 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.zookeeper;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.zookeeper.KeeperException;
+
+
+/**
+ * Tracks the switch of split and merge states in ZK
+ *
+ */
+@InterfaceAudience.Private
+public class SplitOrMergeTrackerManager {
+
+  private String splitZnode;
+  private String mergeZnode;
+
+  private SwitchStateTracker splitStateTracker;
+  private SwitchStateTracker mergeStateTracker;
+
+  public SplitOrMergeTrackerManager(ZooKeeperWatcher watcher, Configuration conf,
+                                    Abortable abortable) {
+    try {
+      if (ZKUtil.checkExists(watcher, watcher.getSwitchZNode()) < 0) {
+        ZKUtil.createAndFailSilent(watcher, watcher.getSwitchZNode());
+      }
+    } catch (KeeperException e) {
+      throw new RuntimeException(e);
+    }
+    splitZnode = ZKUtil.joinZNode(watcher.getSwitchZNode(),
+      conf.get("zookeeper.znode.switch.split", "split"));
+    mergeZnode = ZKUtil.joinZNode(watcher.getSwitchZNode(),
+      conf.get("zookeeper.znode.switch.merge", "merge"));
+    splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
+    mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);
+  }
+
+  public void start() {
+    splitStateTracker.start();
+    mergeStateTracker.start();
+  }
+
+  public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) {
+    switch (switchType) {
+      case SPLIT:
+        return splitStateTracker.isSwitchEnabled();
+      case MERGE:
+        return mergeStateTracker.isSwitchEnabled();
+      default:
+        break;
+    }
+    return false;
+  }
+
+  public void setSplitOrMergeEnabled(boolean enabled, Admin.MasterSwitchType switchType)
+    throws KeeperException {
+    switch (switchType) {
+      case SPLIT:
+        splitStateTracker.setSwitchEnabled(enabled);
+        break;
+      case MERGE:
+        mergeStateTracker.setSwitchEnabled(enabled);
+        break;
+      default:
+        break;
+    }
+  }
+
+  private static class SwitchStateTracker extends ZooKeeperNodeTracker {
+
+    public SwitchStateTracker(ZooKeeperWatcher watcher, String node, Abortable abortable) {
+      super(watcher, node, abortable);
+    }
+
+    /**
+     * Return true if the switch is on, false otherwise
+     */
+    public boolean isSwitchEnabled() {
+      byte [] upData = super.getData(false);
+      try {
+        // if data in ZK is null, use default of on.
+        return upData == null || parseFrom(upData).getEnabled();
+      } catch (DeserializationException dex) {
+        LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData));
+        // return false to be safe.
+        return false;
+      }
+    }
+
+    /**
+     * Set the switch on/off
+     * @param enabled switch enabled or not?
+     * @throws KeeperException keepException will be thrown out
+     */
+    public void setSwitchEnabled(boolean enabled) throws KeeperException {
+      byte [] upData = toByteArray(enabled);
+      try {
+        ZKUtil.setData(watcher, node, upData);
+      } catch(KeeperException.NoNodeException nne) {
+        ZKUtil.createAndWatch(watcher, node, upData);
+      }
+      super.nodeDataChanged(node);
+    }
+
+    private byte [] toByteArray(boolean enabled) {
+      SwitchState.Builder builder = SwitchState.newBuilder();
+      builder.setEnabled(enabled);
+      return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+    }
+
+    private SwitchState parseFrom(byte [] bytes)
+      throws DeserializationException {
+      ProtobufUtil.expectPBMagicPrefix(bytes);
+      SwitchState.Builder builder = SwitchState.newBuilder();
+      try {
+        int magicLen = ProtobufUtil.lengthOfPBMagic();
+        ProtobufUtil.mergeFrom(builder, bytes, magicLen, bytes.length - magicLen);
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+      return builder.build();
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
new file mode 100644
index 0000000..6405a14
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
@@ -0,0 +1,198 @@
+/**
+ * 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.client;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@Category({MediumTests.class, ClientTests.class})
+public class TestSplitOrMergeStatus {
+
+  private static final Log LOG = LogFactory.getLog(TestSplitOrMergeStatus.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static byte [] FAMILY = Bytes.toBytes("testFamily");
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster(2);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testSplitSwitch() throws Exception {
+    TableName name = TableName.valueOf("testSplitSwitch");
+    Table t = TEST_UTIL.createTable(name, FAMILY);
+    TEST_UTIL.loadTable(t, FAMILY, false);
+
+    RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName());
+    int orignalCount = locator.getAllRegionLocations().size();
+
+    Admin admin = TEST_UTIL.getAdmin();
+    initSwitchStatus(admin);
+    boolean[] results = admin.setSplitOrMergeEnabled(false, false, Admin.MasterSwitchType.SPLIT);
+    assertEquals(results.length, 1);
+    assertTrue(results[0]);
+    admin.split(t.getName());
+    int count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount == count);
+
+    results = admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
+    assertEquals(results.length, 1);
+    assertFalse(results[0]);
+    admin.split(t.getName());
+    count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount<count);
+    admin.close();
+  }
+
+
+  @Test
+  public void testMergeSwitch() throws Exception {
+    TableName name = TableName.valueOf("testMergeSwitch");
+    Table t = TEST_UTIL.createTable(name, FAMILY);
+    TEST_UTIL.loadTable(t, FAMILY, false);
+
+    RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName());
+
+    Admin admin = TEST_UTIL.getAdmin();
+    initSwitchStatus(admin);
+    admin.split(t.getName());
+    waitOnSplitOrMerge(t); //Split the table to ensure we have two regions at least.
+
+    waitForMergable(admin, name);
+    int orignalCount = locator.getAllRegionLocations().size();
+    boolean[] results = admin.setSplitOrMergeEnabled(false, false, Admin.MasterSwitchType.MERGE);
+    assertEquals(results.length, 1);
+    assertTrue(results[0]);
+    List<HRegionInfo> regions = admin.getTableRegions(t.getName());
+    assertTrue(regions.size() > 1);
+    admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
+      regions.get(1).getEncodedNameAsBytes(), true);
+    int count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount == count);
+
+    waitForMergable(admin, name);
+    results = admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
+    assertEquals(results.length, 1);
+    assertFalse(results[0]);
+    admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
+      regions.get(1).getEncodedNameAsBytes(), true);
+    count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount>count);
+    admin.close();
+  }
+
+  @Test
+  public void testMultiSwitches() throws IOException {
+    Admin admin = TEST_UTIL.getAdmin();
+    boolean[] switches = admin.setSplitOrMergeEnabled(false, false,
+      Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
+    for (boolean s : switches){
+      assertTrue(s);
+    }
+    assertFalse(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT));
+    assertFalse(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE));
+    admin.close();
+  }
+
+  private void initSwitchStatus(Admin admin) throws IOException {
+    if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT)) {
+      admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
+    }
+    if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE)) {
+      admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
+    }
+    assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT));
+    assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE));
+  }
+
+  private void waitForMergable(Admin admin, TableName t) throws InterruptedException, IOException {
+    // Wait for the Regions to be mergeable
+    MiniHBaseCluster miniCluster = TEST_UTIL.getMiniHBaseCluster();
+    int mergeable = 0;
+    while (mergeable < 2) {
+      Thread.sleep(100);
+      admin.majorCompact(t);
+      mergeable = 0;
+      for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
+        for (Region region: regionThread.getRegionServer().getOnlineRegions(t)) {
+          mergeable += ((HRegion)region).isMergeable() ? 1 : 0;
+        }
+      }
+    }
+  }
+
+  /*
+   * Wait on table split.  May return because we waited long enough on the split
+   * and it didn't happen.  Caller should check.
+   * @param t
+   * @return Map of table regions; caller needs to check table actually split.
+   */
+  private List<HRegionLocation> waitOnSplitOrMerge(final Table t)
+    throws IOException {
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
+      List<HRegionLocation> regions = locator.getAllRegionLocations();
+      int originalCount = regions.size();
+      for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 10); i++) {
+        Thread.currentThread();
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+        regions = locator.getAllRegionLocations();
+        if (regions.size() !=  originalCount)
+          break;
+      }
+      return regions;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 82f0700..40c3711 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -132,6 +132,38 @@ module Hbase
       end
     end
 
+    #----------------------------------------------------------------------------------------------
+    # Enable/disable one split or merge switch
+    # Returns previous switch setting.
+    def splitormerge_switch(type, enabled)
+      switch_type = nil
+      if type == 'SPLIT'
+        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::SPLIT
+      elsif type == 'MERGE'
+        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::MERGE
+      else
+        raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
+      end
+      @admin.setSplitOrMergeEnabled(
+        java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false),
+        switch_type)[0]
+    end
+
+    #----------------------------------------------------------------------------------------------
+    # Query the current state of the split or merge switch.
+    # Returns the switch's state (true is enabled).
+    def splitormerge_enabled(type)
+      switch_type = nil
+      if type == 'SPLIT'
+        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::SPLIT
+      elsif type == 'MERGE'
+        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::MERGE
+      else
+        raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
+      end
+      @admin.isSplitOrMergeEnabled(switch_type)
+    end
+
     def locate_region(table_name, row_key)
       locator = @connection.getRegionLocator(TableName.valueOf(table_name))
       begin

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 0ecd3d7..4144b91 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -333,6 +333,8 @@ Shell.load_command_group(
     catalogjanitor_enabled
     compact_rs
     trace
+    splitormerge_switch
+    splitormerge_enabled
   ],
   # TODO remove older hlog_roll command
   :aliases => {

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb b/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb
new file mode 100644
index 0000000..7da7564
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb
@@ -0,0 +1,41 @@
+#!/usr/bin/env hbase-jruby
+#
+# 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.
+
+# Prints the current split or merge status
+module Shell
+  module Commands
+    # Command for check split or merge switch status
+    class SplitormergeEnabled < Command
+      def help
+        print <<-EOF
+Query the switch's state. You can set switch type, 'SPLIT' or 'MERGE'
+Examples:
+
+  hbase> splitormerge_enabled 'SPLIT'
+EOF
+      end
+
+      def command(switch_type)
+        format_simple_command do
+          formatter.row(
+            [admin.splitormerge_enabled(switch_type) ? 'true' : 'false']
+          )
+        end
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb b/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb
new file mode 100644
index 0000000..f4c2858
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb
@@ -0,0 +1,43 @@
+#
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    # Command for set switch for split and merge
+    class SplitormergeSwitch < Command
+      def help
+        print <<-EOF
+Enable/Disable one switch. You can set switch type 'SPLIT' or 'MERGE'. Returns previous split state.
+Examples:
+
+  hbase> splitormerge_switch 'SPLIT', true
+  hbase> splitormerge_switch 'SPLIT', false
+EOF
+      end
+
+      def command(switch_type, enabled)
+        format_simple_command do
+          formatter.row(
+            [admin.splitormerge_switch(switch_type, enabled) ? 'true' : 'false']
+          )
+        end
+      end
+    end
+  end
+end


[04/37] hbase git commit: HBASE-13259 mmap() based BucketCache IOEngine (Zee Chen & Ram)

Posted by sy...@apache.org.
HBASE-13259 mmap() based BucketCache IOEngine (Zee Chen & Ram)


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

Branch: refs/heads/hbase-12439
Commit: 3ba1a7fd23f0b0ca06cf7a9a04cb45975e1c7d91
Parents: a8073c4
Author: ramkrishna <ra...@gmail.com>
Authored: Tue Feb 23 17:03:38 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Tue Feb 23 17:03:38 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/util/ByteBufferAllocator.java  |  39 +++++
 .../hadoop/hbase/util/ByteBufferArray.java      |  16 +-
 .../hadoop/hbase/util/TestByteBufferArray.java  |  16 +-
 .../hbase/io/hfile/bucket/BucketCache.java      |  11 +-
 .../io/hfile/bucket/ByteBufferIOEngine.java     |  18 +-
 .../hbase/io/hfile/bucket/FileMmapEngine.java   | 166 +++++++++++++++++++
 .../io/hfile/bucket/TestFileMmapEngine.java     |  68 ++++++++
 7 files changed, 320 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3ba1a7fd/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java
new file mode 100644
index 0000000..b19a0a7
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java
@@ -0,0 +1,39 @@
+/**
+ * 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.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Defines the way the ByteBuffers are created
+ */
+@InterfaceAudience.Private
+public interface ByteBufferAllocator {
+
+  /**
+   * Allocates a bytebuffer
+   * @param size the size of the bytebuffer
+   * @param directByteBuffer indicator to create a direct bytebuffer
+   * @return the bytebuffer that is created
+   * @throws IOException exception thrown if there is an error while creating the ByteBuffer
+   */
+  ByteBuffer allocate(long size, boolean directByteBuffer) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ba1a7fd/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
index 2334cf7..b09dc9a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -39,20 +40,23 @@ import org.apache.hadoop.util.StringUtils;
 public final class ByteBufferArray {
   private static final Log LOG = LogFactory.getLog(ByteBufferArray.class);
 
-  static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
+  public static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
   private ByteBuffer buffers[];
   private Lock locks[];
   private int bufferSize;
   private int bufferCount;
-
+  private ByteBufferAllocator allocator;
   /**
    * We allocate a number of byte buffers as the capacity. In order not to out
    * of the array bounds for the last byte(see {@link ByteBufferArray#multiple}),
    * we will allocate one additional buffer with capacity 0;
    * @param capacity total size of the byte buffer array
    * @param directByteBuffer true if we allocate direct buffer
+   * @param allocator the ByteBufferAllocator that will create the buffers
+   * @throws IOException throws IOException if there is an exception thrown by the allocator
    */
-  public ByteBufferArray(long capacity, boolean directByteBuffer) {
+  public ByteBufferArray(long capacity, boolean directByteBuffer, ByteBufferAllocator allocator)
+      throws IOException {
     this.bufferSize = DEFAULT_BUFFER_SIZE;
     if (this.bufferSize > (capacity / 16))
       this.bufferSize = (int) roundUp(capacity / 16, 32768);
@@ -62,15 +66,15 @@ public final class ByteBufferArray {
         + bufferCount + ", direct=" + directByteBuffer);
     buffers = new ByteBuffer[bufferCount + 1];
     locks = new Lock[bufferCount + 1];
+    this.allocator = allocator;
     for (int i = 0; i <= bufferCount; i++) {
       locks[i] = new ReentrantLock();
       if (i < bufferCount) {
-        buffers[i] = directByteBuffer ? ByteBuffer.allocateDirect(bufferSize)
-            : ByteBuffer.allocate(bufferSize);
+        buffers[i] = allocator.allocate(bufferSize, directByteBuffer);
       } else {
+        // always create on heap
         buffers[i] = ByteBuffer.allocate(0);
       }
-
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ba1a7fd/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
index 701601d..f2c8549 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
@@ -20,6 +20,9 @@ package org.apache.hadoop.hbase.util;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -32,7 +35,18 @@ public class TestByteBufferArray {
   @Test
   public void testAsSubBufferWhenEndOffsetLandInLastBuffer() throws Exception {
     int capacity = 4 * 1024 * 1024;
-    ByteBufferArray array = new ByteBufferArray(capacity, false);
+    ByteBufferAllocator allocator = new ByteBufferAllocator() {
+      @Override
+      public ByteBuffer allocate(long size, boolean directByteBuffer)
+          throws IOException {
+        if (directByteBuffer) {
+          return ByteBuffer.allocateDirect((int) size);
+        } else {
+          return ByteBuffer.allocate((int) size);
+        }
+      }
+    };
+    ByteBufferArray array = new ByteBufferArray(capacity, false, allocator);
     ByteBuff subBuf = array.asSubByteBuff(0, capacity);
     subBuf.position(capacity - 1);// Position to the last byte
     assertTrue(subBuf.hasRemaining());

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ba1a7fd/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 6024958..7436b71 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -303,15 +303,18 @@ public class BucketCache implements BlockCache, HeapSize {
    */
   private IOEngine getIOEngineFromName(String ioEngineName, long capacity)
       throws IOException {
-    if (ioEngineName.startsWith("file:"))
+    if (ioEngineName.startsWith("file:")) {
       return new FileIOEngine(ioEngineName.substring(5), capacity);
-    else if (ioEngineName.startsWith("offheap"))
+    } else if (ioEngineName.startsWith("offheap")) {
       return new ByteBufferIOEngine(capacity, true);
-    else if (ioEngineName.startsWith("heap"))
+    } else if (ioEngineName.startsWith("heap")) {
       return new ByteBufferIOEngine(capacity, false);
-    else
+    } else if (ioEngineName.startsWith("mmap:")) {
+      return new FileMmapEngine(ioEngineName.substring(5), capacity);
+    } else {
       throw new IllegalArgumentException(
           "Don't understand io engine name for cache - prefix with file:, heap or offheap");
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ba1a7fd/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
index 2227312..45ed1ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.io.hfile.Cacheable;
 import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
 import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
 import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.util.ByteBufferAllocator;
 import org.apache.hadoop.hbase.util.ByteBufferArray;
 
 /**
@@ -42,13 +43,24 @@ public class ByteBufferIOEngine implements IOEngine {
    * Construct the ByteBufferIOEngine with the given capacity
    * @param capacity
    * @param direct true if allocate direct buffer
-   * @throws IOException
+   * @throws IOException ideally here no exception to be thrown from the allocator
    */
   public ByteBufferIOEngine(long capacity, boolean direct)
       throws IOException {
     this.capacity = capacity;
     this.direct = direct;
-    bufferArray = new ByteBufferArray(capacity, direct);
+    ByteBufferAllocator allocator = new ByteBufferAllocator() {
+      @Override
+      public ByteBuffer allocate(long size, boolean directByteBuffer)
+          throws IOException {
+        if (directByteBuffer) {
+          return ByteBuffer.allocateDirect((int) size);
+        } else {
+          return ByteBuffer.allocate((int) size);
+        }
+      }
+    };
+    bufferArray = new ByteBufferArray(capacity, direct, allocator);
   }
 
   @Override
@@ -85,7 +97,7 @@ public class ByteBufferIOEngine implements IOEngine {
    * @param srcBuffer the given byte buffer from which bytes are to be read
    * @param offset The offset in the ByteBufferArray of the first byte to be
    *          written
-   * @throws IOException
+   * @throws IOException throws IOException if writing to the array throws exception
    */
   @Override
   public void write(ByteBuffer srcBuffer, long offset) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ba1a7fd/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
new file mode 100644
index 0000000..7a2afe8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
@@ -0,0 +1,166 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.io.hfile.bucket;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.hfile.Cacheable;
+import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
+import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.nio.SingleByteBuff;
+import org.apache.hadoop.hbase.util.ByteBufferAllocator;
+import org.apache.hadoop.hbase.util.ByteBufferArray;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * IO engine that stores data to a file on the local file system using memory mapping
+ * mechanism
+ */
+@InterfaceAudience.Private
+public class FileMmapEngine implements IOEngine {
+  static final Log LOG = LogFactory.getLog(FileMmapEngine.class);
+
+  private final String path;
+  private long size;
+  private ByteBufferArray bufferArray;
+  private final FileChannel fileChannel;
+  private RandomAccessFile raf = null;
+
+  public FileMmapEngine(String filePath, long capacity) throws IOException {
+    this.path = filePath;
+    this.size = capacity;
+    long fileSize = 0;
+    try {
+      raf = new RandomAccessFile(filePath, "rw");
+      fileSize = roundUp(capacity, ByteBufferArray.DEFAULT_BUFFER_SIZE);
+      raf.setLength(fileSize);
+      fileChannel = raf.getChannel();
+      LOG.info("Allocating " + StringUtils.byteDesc(fileSize) + ", on the path:" + filePath);
+    } catch (java.io.FileNotFoundException fex) {
+      LOG.error("Can't create bucket cache file " + filePath, fex);
+      throw fex;
+    } catch (IOException ioex) {
+      LOG.error("Can't extend bucket cache file; insufficient space for "
+          + StringUtils.byteDesc(fileSize), ioex);
+      shutdown();
+      throw ioex;
+    }
+    ByteBufferAllocator allocator = new ByteBufferAllocator() {
+      int pos = 0;
+      @Override
+      public ByteBuffer allocate(long size, boolean directByteBuffer) throws IOException {
+        ByteBuffer buffer = null;
+        if (directByteBuffer) {
+          buffer = fileChannel.map(java.nio.channels.FileChannel.MapMode.READ_WRITE, pos * size,
+              size);
+        } else {
+          throw new IllegalArgumentException(
+              "Only Direct Bytebuffers allowed with FileMMap engine");
+        }
+        pos++;
+        return buffer;
+      }
+    };
+    bufferArray = new ByteBufferArray(fileSize, true, allocator);
+  }
+
+  private long roundUp(long n, long to) {
+    return ((n + to - 1) / to) * to;
+  }
+
+  @Override
+  public String toString() {
+    return "ioengine=" + this.getClass().getSimpleName() + ", path=" + this.path +
+      ", size=" + String.format("%,d", this.size);
+  }
+
+  /**
+   * File IO engine is always able to support persistent storage for the cache
+   * @return true
+   */
+  @Override
+  public boolean isPersistent() {
+    return true;
+  }
+
+  @Override
+  public Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
+      throws IOException {
+    byte[] dst = new byte[length];
+    bufferArray.getMultiple(offset, length, dst);
+    return deserializer.deserialize(new SingleByteBuff(ByteBuffer.wrap(dst)), true,
+        MemoryType.EXCLUSIVE);
+  }
+
+  /**
+   * Transfers data from the given byte buffer to file
+   * @param srcBuffer the given byte buffer from which bytes are to be read
+   * @param offset The offset in the file where the first byte to be written
+   * @throws IOException
+   */
+  @Override
+  public void write(ByteBuffer srcBuffer, long offset) throws IOException {
+    assert srcBuffer.hasArray();
+    bufferArray.putMultiple(offset, srcBuffer.remaining(), srcBuffer.array(),
+        srcBuffer.arrayOffset());
+  }
+
+  @Override
+  public void write(ByteBuff srcBuffer, long offset) throws IOException {
+    // This singleByteBuff can be considered to be array backed
+    assert srcBuffer.hasArray();
+    bufferArray.putMultiple(offset, srcBuffer.remaining(), srcBuffer.array(),
+        srcBuffer.arrayOffset());
+
+  }
+  /**
+   * Sync the data to file after writing
+   * @throws IOException
+   */
+  @Override
+  public void sync() throws IOException {
+    if (fileChannel != null) {
+      fileChannel.force(true);
+    }
+  }
+
+  /**
+   * Close the file
+   */
+  @Override
+  public void shutdown() {
+    try {
+      fileChannel.close();
+    } catch (IOException ex) {
+      LOG.error("Can't shutdown cleanly", ex);
+    }
+    try {
+      raf.close();
+    } catch (IOException ex) {
+      LOG.error("Can't shutdown cleanly", ex);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ba1a7fd/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java
new file mode 100644
index 0000000..dfc18c7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java
@@ -0,0 +1,68 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.io.hfile.bucket;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.io.hfile.bucket.TestByteBufferIOEngine.BufferGrabbingDeserializer;
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Basic test for {@link FileMmapEngine}
+ */
+@Category({IOTests.class, SmallTests.class})
+public class TestFileMmapEngine {
+  @Test
+  public void testFileMmapEngine() throws IOException {
+    int size = 2 * 1024 * 1024; // 2 MB
+    String filePath = "testFileMmapEngine";
+    try {
+      FileMmapEngine fileMmapEngine = new FileMmapEngine(filePath, size);
+      for (int i = 0; i < 50; i++) {
+        int len = (int) Math.floor(Math.random() * 100);
+        long offset = (long) Math.floor(Math.random() * size % (size - len));
+        byte[] data1 = new byte[len];
+        for (int j = 0; j < data1.length; ++j) {
+          data1[j] = (byte) (Math.random() * 255);
+        }
+        fileMmapEngine.write(ByteBuffer.wrap(data1), offset);
+        BufferGrabbingDeserializer deserializer = new BufferGrabbingDeserializer();
+        fileMmapEngine.read(offset, len, deserializer);
+        ByteBuff data2 = deserializer.getDeserializedByteBuff();
+        for (int j = 0; j < data1.length; ++j) {
+          assertTrue(data1[j] == data2.get(j));
+        }
+      }
+    } finally {
+      File file = new File(filePath);
+      if (file.exists()) {
+        file.delete();
+      }
+    }
+
+  }
+}
\ No newline at end of file


[24/37] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/24d481c5/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
index 8dbb5ad..9805d50 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
@@ -11,13 +11,13 @@ public final class SnapshotProtos {
   public interface SnapshotFileInfoOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .SnapshotFileInfo.Type type = 1;
+    // required .hbase.pb.SnapshotFileInfo.Type type = 1;
     /**
-     * <code>required .SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
      */
     boolean hasType();
     /**
-     * <code>required .SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType();
 
@@ -67,7 +67,7 @@ public final class SnapshotProtos {
         getWalNameBytes();
   }
   /**
-   * Protobuf type {@code SnapshotFileInfo}
+   * Protobuf type {@code hbase.pb.SnapshotFileInfo}
    */
   public static final class SnapshotFileInfo extends
       com.google.protobuf.GeneratedMessage
@@ -157,12 +157,12 @@ public final class SnapshotProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Builder.class);
     }
@@ -183,7 +183,7 @@ public final class SnapshotProtos {
     }
 
     /**
-     * Protobuf enum {@code SnapshotFileInfo.Type}
+     * Protobuf enum {@code hbase.pb.SnapshotFileInfo.Type}
      */
     public enum Type
         implements com.google.protobuf.ProtocolMessageEnum {
@@ -261,21 +261,21 @@ public final class SnapshotProtos {
         this.value = value;
       }
 
-      // @@protoc_insertion_point(enum_scope:SnapshotFileInfo.Type)
+      // @@protoc_insertion_point(enum_scope:hbase.pb.SnapshotFileInfo.Type)
     }
 
     private int bitField0_;
-    // required .SnapshotFileInfo.Type type = 1;
+    // required .hbase.pb.SnapshotFileInfo.Type type = 1;
     public static final int TYPE_FIELD_NUMBER = 1;
     private org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type type_;
     /**
-     * <code>required .SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
      */
     public boolean hasType() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required .SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType() {
       return type_;
@@ -613,19 +613,19 @@ public final class SnapshotProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code SnapshotFileInfo}
+     * Protobuf type {@code hbase.pb.SnapshotFileInfo}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
        implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfoOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Builder.class);
       }
@@ -667,7 +667,7 @@ public final class SnapshotProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
       }
 
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo getDefaultInstanceForType() {
@@ -767,22 +767,22 @@ public final class SnapshotProtos {
       }
       private int bitField0_;
 
-      // required .SnapshotFileInfo.Type type = 1;
+      // required .hbase.pb.SnapshotFileInfo.Type type = 1;
       private org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type type_ = org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type.HFILE;
       /**
-       * <code>required .SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
        */
       public boolean hasType() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required .SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType() {
         return type_;
       }
       /**
-       * <code>required .SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
        */
       public Builder setType(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type value) {
         if (value == null) {
@@ -794,7 +794,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
        */
       public Builder clearType() {
         bitField0_ = (bitField0_ & ~0x00000001);
@@ -1025,7 +1025,7 @@ public final class SnapshotProtos {
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:SnapshotFileInfo)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotFileInfo)
     }
 
     static {
@@ -1033,7 +1033,7 @@ public final class SnapshotProtos {
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:SnapshotFileInfo)
+    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotFileInfo)
   }
 
   public interface SnapshotRegionManifestOrBuilder
@@ -1049,47 +1049,47 @@ public final class SnapshotProtos {
      */
     int getVersion();
 
-    // required .RegionInfo region_info = 2;
+    // required .hbase.pb.RegionInfo region_info = 2;
     /**
-     * <code>required .RegionInfo region_info = 2;</code>
+     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
      */
     boolean hasRegionInfo();
     /**
-     * <code>required .RegionInfo region_info = 2;</code>
+     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo();
     /**
-     * <code>required .RegionInfo region_info = 2;</code>
+     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder();
 
-    // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;
+    // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> 
         getFamilyFilesList();
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index);
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     int getFamilyFilesCount();
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> 
         getFamilyFilesOrBuilderList();
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder(
         int index);
   }
   /**
-   * Protobuf type {@code SnapshotRegionManifest}
+   * Protobuf type {@code hbase.pb.SnapshotRegionManifest}
    */
   public static final class SnapshotRegionManifest extends
       com.google.protobuf.GeneratedMessage
@@ -1182,12 +1182,12 @@ public final class SnapshotProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder.class);
     }
@@ -1225,17 +1225,17 @@ public final class SnapshotProtos {
       com.google.protobuf.ByteString
           getNameBytes();
 
-      // optional .Reference reference = 2;
+      // optional .hbase.pb.Reference reference = 2;
       /**
-       * <code>optional .Reference reference = 2;</code>
+       * <code>optional .hbase.pb.Reference reference = 2;</code>
        */
       boolean hasReference();
       /**
-       * <code>optional .Reference reference = 2;</code>
+       * <code>optional .hbase.pb.Reference reference = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference();
       /**
-       * <code>optional .Reference reference = 2;</code>
+       * <code>optional .hbase.pb.Reference reference = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder();
 
@@ -1258,7 +1258,7 @@ public final class SnapshotProtos {
       long getFileSize();
     }
     /**
-     * Protobuf type {@code SnapshotRegionManifest.StoreFile}
+     * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile}
      */
     public static final class StoreFile extends
         com.google.protobuf.GeneratedMessage
@@ -1345,12 +1345,12 @@ public final class SnapshotProtos {
       }
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder.class);
       }
@@ -1414,23 +1414,23 @@ public final class SnapshotProtos {
         }
       }
 
-      // optional .Reference reference = 2;
+      // optional .hbase.pb.Reference reference = 2;
       public static final int REFERENCE_FIELD_NUMBER = 2;
       private org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference reference_;
       /**
-       * <code>optional .Reference reference = 2;</code>
+       * <code>optional .hbase.pb.Reference reference = 2;</code>
        */
       public boolean hasReference() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional .Reference reference = 2;</code>
+       * <code>optional .hbase.pb.Reference reference = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference() {
         return reference_;
       }
       /**
-       * <code>optional .Reference reference = 2;</code>
+       * <code>optional .hbase.pb.Reference reference = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder() {
         return reference_;
@@ -1652,19 +1652,19 @@ public final class SnapshotProtos {
         return builder;
       }
       /**
-       * Protobuf type {@code SnapshotRegionManifest.StoreFile}
+       * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile}
        */
       public static final class Builder extends
           com.google.protobuf.GeneratedMessage.Builder<Builder>
          implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder {
         public static final com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
         }
 
         protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
             internalGetFieldAccessorTable() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
                   org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder.class);
         }
@@ -1709,7 +1709,7 @@ public final class SnapshotProtos {
 
         public com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
         }
 
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getDefaultInstanceForType() {
@@ -1882,18 +1882,18 @@ public final class SnapshotProtos {
           return this;
         }
 
-        // optional .Reference reference = 2;
+        // optional .hbase.pb.Reference reference = 2;
         private org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference reference_ = org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.getDefaultInstance();
         private com.google.protobuf.SingleFieldBuilder<
             org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder> referenceBuilder_;
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public boolean hasReference() {
           return ((bitField0_ & 0x00000002) == 0x00000002);
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference() {
           if (referenceBuilder_ == null) {
@@ -1903,7 +1903,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public Builder setReference(org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference value) {
           if (referenceBuilder_ == null) {
@@ -1919,7 +1919,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public Builder setReference(
             org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder builderForValue) {
@@ -1933,7 +1933,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public Builder mergeReference(org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference value) {
           if (referenceBuilder_ == null) {
@@ -1952,7 +1952,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public Builder clearReference() {
           if (referenceBuilder_ == null) {
@@ -1965,7 +1965,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder getReferenceBuilder() {
           bitField0_ |= 0x00000002;
@@ -1973,7 +1973,7 @@ public final class SnapshotProtos {
           return getReferenceFieldBuilder().getBuilder();
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder() {
           if (referenceBuilder_ != null) {
@@ -1983,7 +1983,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>optional .Reference reference = 2;</code>
+         * <code>optional .hbase.pb.Reference reference = 2;</code>
          */
         private com.google.protobuf.SingleFieldBuilder<
             org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder> 
@@ -2048,7 +2048,7 @@ public final class SnapshotProtos {
           return this;
         }
 
-        // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest.StoreFile)
+        // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest.StoreFile)
       }
 
       static {
@@ -2056,7 +2056,7 @@ public final class SnapshotProtos {
         defaultInstance.initFields();
       }
 
-      // @@protoc_insertion_point(class_scope:SnapshotRegionManifest.StoreFile)
+      // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest.StoreFile)
     }
 
     public interface FamilyFilesOrBuilder
@@ -2072,33 +2072,33 @@ public final class SnapshotProtos {
        */
       com.google.protobuf.ByteString getFamilyName();
 
-      // repeated .SnapshotRegionManifest.StoreFile store_files = 2;
+      // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> 
           getStoreFilesList();
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index);
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       int getStoreFilesCount();
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> 
           getStoreFilesOrBuilderList();
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder(
           int index);
     }
     /**
-     * Protobuf type {@code SnapshotRegionManifest.FamilyFiles}
+     * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles}
      */
     public static final class FamilyFiles extends
         com.google.protobuf.GeneratedMessage
@@ -2178,12 +2178,12 @@ public final class SnapshotProtos {
       }
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder.class);
       }
@@ -2220,36 +2220,36 @@ public final class SnapshotProtos {
         return familyName_;
       }
 
-      // repeated .SnapshotRegionManifest.StoreFile store_files = 2;
+      // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;
       public static final int STORE_FILES_FIELD_NUMBER = 2;
       private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> storeFiles_;
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> getStoreFilesList() {
         return storeFiles_;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> 
           getStoreFilesOrBuilderList() {
         return storeFiles_;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public int getStoreFilesCount() {
         return storeFiles_.size();
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index) {
         return storeFiles_.get(index);
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder(
           int index) {
@@ -2428,19 +2428,19 @@ public final class SnapshotProtos {
         return builder;
       }
       /**
-       * Protobuf type {@code SnapshotRegionManifest.FamilyFiles}
+       * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles}
        */
       public static final class Builder extends
           com.google.protobuf.GeneratedMessage.Builder<Builder>
          implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder {
         public static final com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
         }
 
         protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
             internalGetFieldAccessorTable() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
                   org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder.class);
         }
@@ -2483,7 +2483,7 @@ public final class SnapshotProtos {
 
         public com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
         }
 
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getDefaultInstanceForType() {
@@ -2633,7 +2633,7 @@ public final class SnapshotProtos {
           return this;
         }
 
-        // repeated .SnapshotRegionManifest.StoreFile store_files = 2;
+        // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;
         private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> storeFiles_ =
           java.util.Collections.emptyList();
         private void ensureStoreFilesIsMutable() {
@@ -2647,7 +2647,7 @@ public final class SnapshotProtos {
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> storeFilesBuilder_;
 
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> getStoreFilesList() {
           if (storeFilesBuilder_ == null) {
@@ -2657,7 +2657,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public int getStoreFilesCount() {
           if (storeFilesBuilder_ == null) {
@@ -2667,7 +2667,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index) {
           if (storeFilesBuilder_ == null) {
@@ -2677,7 +2677,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder setStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) {
@@ -2694,7 +2694,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder setStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) {
@@ -2708,7 +2708,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) {
           if (storeFilesBuilder_ == null) {
@@ -2724,7 +2724,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) {
@@ -2741,7 +2741,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) {
@@ -2755,7 +2755,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) {
@@ -2769,7 +2769,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addAllStoreFiles(
             java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> values) {
@@ -2783,7 +2783,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder clearStoreFiles() {
           if (storeFilesBuilder_ == null) {
@@ -2796,7 +2796,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder removeStoreFiles(int index) {
           if (storeFilesBuilder_ == null) {
@@ -2809,14 +2809,14 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder getStoreFilesBuilder(
             int index) {
           return getStoreFilesFieldBuilder().getBuilder(index);
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder(
             int index) {
@@ -2826,7 +2826,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> 
              getStoreFilesOrBuilderList() {
@@ -2837,14 +2837,14 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder addStoreFilesBuilder() {
           return getStoreFilesFieldBuilder().addBuilder(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.getDefaultInstance());
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder addStoreFilesBuilder(
             int index) {
@@ -2852,7 +2852,7 @@ public final class SnapshotProtos {
               index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.getDefaultInstance());
         }
         /**
-         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder> 
              getStoreFilesBuilderList() {
@@ -2873,7 +2873,7 @@ public final class SnapshotProtos {
           return storeFilesBuilder_;
         }
 
-        // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest.FamilyFiles)
+        // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest.FamilyFiles)
       }
 
       static {
@@ -2881,7 +2881,7 @@ public final class SnapshotProtos {
         defaultInstance.initFields();
       }
 
-      // @@protoc_insertion_point(class_scope:SnapshotRegionManifest.FamilyFiles)
+      // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest.FamilyFiles)
     }
 
     private int bitField0_;
@@ -2901,58 +2901,58 @@ public final class SnapshotProtos {
       return version_;
     }
 
-    // required .RegionInfo region_info = 2;
+    // required .hbase.pb.RegionInfo region_info = 2;
     public static final int REGION_INFO_FIELD_NUMBER = 2;
     private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_;
     /**
-     * <code>required .RegionInfo region_info = 2;</code>
+     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
      */
     public boolean hasRegionInfo() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>required .RegionInfo region_info = 2;</code>
+     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
       return regionInfo_;
     }
     /**
-     * <code>required .RegionInfo region_info = 2;</code>
+     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
       return regionInfo_;
     }
 
-    // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;
+    // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;
     public static final int FAMILY_FILES_FIELD_NUMBER = 3;
     private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> familyFiles_;
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> getFamilyFilesList() {
       return familyFiles_;
     }
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> 
         getFamilyFilesOrBuilderList() {
       return familyFiles_;
     }
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public int getFamilyFilesCount() {
       return familyFiles_.size();
     }
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index) {
       return familyFiles_.get(index);
     }
     /**
-     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder(
         int index) {
@@ -3152,19 +3152,19 @@ public final class SnapshotProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code SnapshotRegionManifest}
+     * Protobuf type {@code hbase.pb.SnapshotRegionManifest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
        implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder.class);
       }
@@ -3214,7 +3214,7 @@ public final class SnapshotProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
       }
 
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getDefaultInstanceForType() {
@@ -3376,18 +3376,18 @@ public final class SnapshotProtos {
         return this;
       }
 
-      // required .RegionInfo region_info = 2;
+      // required .hbase.pb.RegionInfo region_info = 2;
       private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public boolean hasRegionInfo() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
         if (regionInfoBuilder_ == null) {
@@ -3397,7 +3397,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public Builder setRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
         if (regionInfoBuilder_ == null) {
@@ -3413,7 +3413,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public Builder setRegionInfo(
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
@@ -3427,7 +3427,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public Builder mergeRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
         if (regionInfoBuilder_ == null) {
@@ -3446,7 +3446,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public Builder clearRegionInfo() {
         if (regionInfoBuilder_ == null) {
@@ -3459,7 +3459,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() {
         bitField0_ |= 0x00000002;
@@ -3467,7 +3467,7 @@ public final class SnapshotProtos {
         return getRegionInfoFieldBuilder().getBuilder();
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
         if (regionInfoBuilder_ != null) {
@@ -3477,7 +3477,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .RegionInfo region_info = 2;</code>
+       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
@@ -3493,7 +3493,7 @@ public final class SnapshotProtos {
         return regionInfoBuilder_;
       }
 
-      // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;
+      // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;
       private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> familyFiles_ =
         java.util.Collections.emptyList();
       private void ensureFamilyFilesIsMutable() {
@@ -3507,7 +3507,7 @@ public final class SnapshotProtos {
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> familyFilesBuilder_;
 
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> getFamilyFilesList() {
         if (familyFilesBuilder_ == null) {
@@ -3517,7 +3517,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public int getFamilyFilesCount() {
         if (familyFilesBuilder_ == null) {
@@ -3527,7 +3527,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index) {
         if (familyFilesBuilder_ == null) {
@@ -3537,7 +3537,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder setFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) {
@@ -3554,7 +3554,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder setFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) {
@@ -3568,7 +3568,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) {
         if (familyFilesBuilder_ == null) {
@@ -3584,7 +3584,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) {
@@ -3601,7 +3601,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) {
@@ -3615,7 +3615,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) {
@@ -3629,7 +3629,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addAllFamilyFiles(
           java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> values) {
@@ -3643,7 +3643,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder clearFamilyFiles() {
         if (familyFilesBuilder_ == null) {
@@ -3656,7 +3656,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder removeFamilyFiles(int index) {
         if (familyFilesBuilder_ == null) {
@@ -3669,14 +3669,14 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder getFamilyFilesBuilder(
           int index) {
         return getFamilyFilesFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder(
           int index) {
@@ -3686,7 +3686,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> 
            getFamilyFilesOrBuilderList() {
@@ -3697,14 +3697,14 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder addFamilyFilesBuilder() {
         return getFamilyFilesFieldBuilder().addBuilder(
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.getDefaultInstance());
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder addFamilyFilesBuilder(
           int index) {
@@ -3712,7 +3712,7 @@ public final class SnapshotProtos {
             index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.getDefaultInstance());
       }
       /**
-       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder> 
            getFamilyFilesBuilderList() {
@@ -3733,7 +3733,7 @@ public final class SnapshotProtos {
         return familyFilesBuilder_;
       }
 
-      // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest)
     }
 
     static {
@@ -3741,53 +3741,53 @@ public final class SnapshotProtos {
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:SnapshotRegionManifest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest)
   }
 
   public interface SnapshotDataManifestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .TableSchema table_schema = 1;
+    // required .hbase.pb.TableSchema table_schema = 1;
     /**
-     * <code>required .TableSchema table_schema = 1;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
      */
     boolean hasTableSchema();
     /**
-     * <code>required .TableSchema table_schema = 1;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema();
     /**
-     * <code>required .TableSchema table_schema = 1;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder();
 
-    // repeated .SnapshotRegionManifest region_manifests = 2;
+    // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> 
         getRegionManifestsList();
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index);
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     int getRegionManifestsCount();
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> 
         getRegionManifestsOrBuilderList();
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder(
         int index);
   }
   /**
-   * Protobuf type {@code SnapshotDataManifest}
+   * Protobuf type {@code hbase.pb.SnapshotDataManifest}
    */
   public static final class SnapshotDataManifest extends
       com.google.protobuf.GeneratedMessage
@@ -3875,12 +3875,12 @@ public final class SnapshotProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.Builder.class);
     }
@@ -3901,58 +3901,58 @@ public final class SnapshotProtos {
     }
 
     private int bitField0_;
-    // required .TableSchema table_schema = 1;
+    // required .hbase.pb.TableSchema table_schema = 1;
     public static final int TABLE_SCHEMA_FIELD_NUMBER = 1;
     private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_;
     /**
-     * <code>required .TableSchema table_schema = 1;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
      */
     public boolean hasTableSchema() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required .TableSchema table_schema = 1;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
       return tableSchema_;
     }
     /**
-     * <code>required .TableSchema table_schema = 1;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
       return tableSchema_;
     }
 
-    // repeated .SnapshotRegionManifest region_manifests = 2;
+    // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;
     public static final int REGION_MANIFESTS_FIELD_NUMBER = 2;
     private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> regionManifests_;
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> getRegionManifestsList() {
       return regionManifests_;
     }
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> 
         getRegionManifestsOrBuilderList() {
       return regionManifests_;
     }
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     public int getRegionManifestsCount() {
       return regionManifests_.size();
     }
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index) {
       return regionManifests_.get(index);
     }
     /**
-     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder(
         int index) {
@@ -4135,19 +4135,19 @@ public final class SnapshotProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code SnapshotDataManifest}
+     * Protobuf type {@code hbase.pb.SnapshotDataManifest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
        implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.Builder.class);
       }
@@ -4195,7 +4195,7 @@ public final class SnapshotProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
       }
 
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest getDefaultInstanceForType() {
@@ -4317,18 +4317,18 @@ public final class SnapshotProtos {
       }
       private int bitField0_;
 
-      // required .TableSchema table_schema = 1;
+      // required .hbase.pb.TableSchema table_schema = 1;
       private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public boolean hasTableSchema() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
         if (tableSchemaBuilder_ == null) {
@@ -4338,7 +4338,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public Builder setTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
         if (tableSchemaBuilder_ == null) {
@@ -4354,7 +4354,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public Builder setTableSchema(
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
@@ -4368,7 +4368,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public Builder mergeTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
         if (tableSchemaBuilder_ == null) {
@@ -4387,7 +4387,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public Builder clearTableSchema() {
         if (tableSchemaBuilder_ == null) {
@@ -4400,7 +4400,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getTableSchemaBuilder() {
         bitField0_ |= 0x00000001;
@@ -4408,7 +4408,7 @@ public final class SnapshotProtos {
         return getTableSchemaFieldBuilder().getBuilder();
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
         if (tableSchemaBuilder_ != null) {
@@ -4418,7 +4418,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .TableSchema table_schema = 1;</code>
+       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
@@ -4434,7 +4434,7 @@ public final class SnapshotProtos {
         return tableSchemaBuilder_;
       }
 
-      // repeated .SnapshotRegionManifest region_manifests = 2;
+      // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;
       private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> regionManifests_ =
         java.util.Collections.emptyList();
       private void ensureRegionManifestsIsMutable() {
@@ -4448,7 +4448,7 @@ public final class SnapshotProtos {
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> regionManifestsBuilder_;
 
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> getRegionManifestsList() {
         if (regionManifestsBuilder_ == null) {
@@ -4458,7 +4458,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public int getRegionManifestsCount() {
         if (regionManifestsBuilder_ == null) {
@@ -4468,7 +4468,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index) {
         if (regionManifestsBuilder_ == null) {
@@ -4478,7 +4478,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder setRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) {
@@ -4495,7 +4495,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder setRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) {
@@ -4509,7 +4509,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) {
         if (regionManifestsBuilder_ == null) {
@@ -4525,7 +4525,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) {
@@ -4542,7 +4542,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) {
@@ -4556,7 +4556,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) {
@@ -4570,7 +4570,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addAllRegionManifests(
           java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> values) {
@@ -4584,7 +4584,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder clearRegionManifests() {
         if (regionManifestsBuilder_ == null) {
@@ -4597,7 +4597,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder removeRegionManifests(int index) {
         if (regionManifestsBuilder_ == null) {
@@ -4610,14 +4610,14 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder getRegionManifestsBuilder(
           int index) {
         return getRegionManifestsFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder(
           int index) {
@@ -4627,7 +4627,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> 
            getRegionManifestsOrBuilderList() {
@@ -4638,14 +4638,14 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder addRegionManifestsBuilder() {
         return getRegionManifestsFieldBuilder().addBuilder(
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.getDefaultInstance());
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder addRegionManifestsBuilder(
           int index) {
@@ -4653,7 +4653,7 @@ public final class SnapshotProtos {
             index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.getDefaultInstance());
       }
       /**
-       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder> 
            getRegionManifestsBuilderList() {
@@ -4674,7 +4674,7 @@ public final class SnapshotProtos {
         return regionManifestsBuilder_;
       }
 
-      // @@protoc_insertion_point(builder_scope:SnapshotDataManifest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotDataManifest)
     }
 
     static {
@@ -4682,34 +4682,34 @@ public final class SnapshotProtos {
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:SnapshotDataManifest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotDataManifest)
   }
 
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_SnapshotFileInfo_descriptor;
+    internal_static_hbase_pb_SnapshotFileInfo_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_SnapshotFileInfo_fieldAccessorTable;
+      internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_SnapshotRegionManifest_descriptor;
+    internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_SnapshotRegionManifest_fieldAccessorTable;
+      internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_SnapshotRegionManifest_StoreFile_descriptor;
+    internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable;
+      internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
+    internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable;
+      internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_SnapshotDataManifest_descriptor;
+    internal_static_hbase_pb_SnapshotDataManifest_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_SnapshotDataManifest_fieldAccessorTable;
+      internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -4719,58 +4719,60 @@ public final class SnapshotProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\016Snapshot.proto\032\010FS.proto\032\013HBase.proto\"" +
-      "\211\001\n\020SnapshotFileInfo\022$\n\004type\030\001 \002(\0162\026.Sna" +
-      "pshotFileInfo.Type\022\r\n\005hfile\030\003 \001(\t\022\022\n\nwal" +
-      "_server\030\004 \001(\t\022\020\n\010wal_name\030\005 \001(\t\"\032\n\004Type\022" +
-      "\t\n\005HFILE\020\001\022\007\n\003WAL\020\002\"\257\002\n\026SnapshotRegionMa" +
-      "nifest\022\017\n\007version\030\001 \001(\005\022 \n\013region_info\030\002" +
-      " \002(\0132\013.RegionInfo\0229\n\014family_files\030\003 \003(\0132" +
-      "#.SnapshotRegionManifest.FamilyFiles\032K\n\t" +
-      "StoreFile\022\014\n\004name\030\001 \002(\t\022\035\n\treference\030\002 \001" +
-      "(\0132\n.Reference\022\021\n\tfile_size\030\003 \001(\004\032Z\n\013Fam",
-      "ilyFiles\022\023\n\013family_name\030\001 \002(\014\0226\n\013store_f" +
-      "iles\030\002 \003(\0132!.SnapshotRegionManifest.Stor" +
-      "eFile\"m\n\024SnapshotDataManifest\022\"\n\014table_s" +
-      "chema\030\001 \002(\0132\014.TableSchema\0221\n\020region_mani" +
-      "fests\030\002 \003(\0132\027.SnapshotRegionManifestBD\n*" +
-      "org.apache.hadoop.hbase.protobuf.generat" +
-      "edB\016SnapshotProtosH\001\210\001\001\240\001\001"
+      "\n\016Snapshot.proto\022\010hbase.pb\032\010FS.proto\032\013HB" +
+      "ase.proto\"\222\001\n\020SnapshotFileInfo\022-\n\004type\030\001" +
+      " \002(\0162\037.hbase.pb.SnapshotFileInfo.Type\022\r\n" +
+      "\005hfile\030\003 \001(\t\022\022\n\nwal_server\030\004 \001(\t\022\020\n\010wal_" +
+      "name\030\005 \001(\t\"\032\n\004Type\022\t\n\005HFILE\020\001\022\007\n\003WAL\020\002\"\323" +
+      "\002\n\026SnapshotRegionManifest\022\017\n\007version\030\001 \001" +
+      "(\005\022)\n\013region_info\030\002 \002(\0132\024.hbase.pb.Regio" +
+      "nInfo\022B\n\014family_files\030\003 \003(\0132,.hbase.pb.S" +
+      "napshotRegionManifest.FamilyFiles\032T\n\tSto" +
+      "reFile\022\014\n\004name\030\001 \002(\t\022&\n\treference\030\002 \001(\0132",
+      "\023.hbase.pb.Reference\022\021\n\tfile_size\030\003 \001(\004\032" +
+      "c\n\013FamilyFiles\022\023\n\013family_name\030\001 \002(\014\022?\n\013s" +
+      "tore_files\030\002 \003(\0132*.hbase.pb.SnapshotRegi" +
+      "onManifest.StoreFile\"\177\n\024SnapshotDataMani" +
+      "fest\022+\n\014table_schema\030\001 \002(\0132\025.hbase.pb.Ta" +
+      "bleSchema\022:\n\020region_manifests\030\002 \003(\0132 .hb" +
+      "ase.pb.SnapshotRegionManifestBD\n*org.apa" +
+      "che.hadoop.hbase.protobuf.generatedB\016Sna" +
+      "pshotProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
         public com.google.protobuf.ExtensionRegistry assignDescriptors(
             com.google.protobuf.Descriptors.FileDescriptor root) {
           descriptor = root;
-          internal_static_SnapshotFileInfo_descriptor =
+          internal_static_hbase_pb_SnapshotFileInfo_descriptor =
             getDescriptor().getMessageTypes().get(0);
-          internal_static_SnapshotFileInfo_fieldAccessorTable = new
+          internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_SnapshotFileInfo_descriptor,
+              internal_static_hbase_pb_SnapshotFileInfo_descriptor,
               new java.lang.String[] { "Type", "Hfile", "WalServer", "WalName", });
-          internal_static_SnapshotRegionManifest_descriptor =
+          internal_static_hbase_pb_SnapshotRegionManifest_descriptor =
             getDescriptor().getMessageTypes().get(1);
-          internal_static_SnapshotRegionManifest_fieldAccessorTable = new
+          internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_SnapshotRegionManifest_descriptor,
+              internal_static_hbase_pb_SnapshotRegionManifest_descriptor,
               new java.lang.String[] { "Version", "RegionInfo", "FamilyFiles", });
-          internal_static_SnapshotRegionManifest_StoreFile_descriptor =
-            internal_static_SnapshotRegionManifest_descriptor.getNestedTypes().get(0);
-          internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable = new
+          internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor =
+            internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(0);
+          internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_SnapshotRegionManifest_StoreFile_descriptor,
+              internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor,
               new java.lang.String[] { "Name", "Reference", "FileSize", });
-          internal_static_SnapshotRegionManifest_FamilyFiles_descriptor =
-            internal_static_SnapshotRegionManifest_descriptor.getNestedTypes().get(1);
-          internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable = new
+          internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor =
+            internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(1);
+          internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_SnapshotRegionManifest_FamilyFiles_descriptor,
+              internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor,
            

<TRUNCATED>

[10/37] hbase git commit: HBASE-15302 Reenable the other tests disabled by HBASE-14678

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
new file mode 100644
index 0000000..125f5a1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -0,0 +1,514 @@
+/**
+ * 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.master.procedure;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, LargeTests.class})
+public class TestMasterFailoverWithProcedures {
+  private static final Log LOG = LogFactory.getLog(TestMasterFailoverWithProcedures.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static void setupConf(Configuration conf) {
+    // don't waste time retrying with the roll, the test is already slow enough.
+    conf.setInt("hbase.procedure.store.wal.max.retries.before.roll", 1);
+    conf.setInt("hbase.procedure.store.wal.wait.before.roll", 0);
+    conf.setInt("hbase.procedure.store.wal.max.roll.retries", 1);
+    conf.setInt("hbase.procedure.store.wal.sync.failure.roll.max", 1);
+  }
+
+  @Before
+  public void setup() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(2, 1);
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, false);
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, false);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testWalRecoverLease() throws Exception {
+    final ProcedureStore masterStore = getMasterProcedureExecutor().getStore();
+    assertTrue("expected WALStore for this test", masterStore instanceof WALProcedureStore);
+
+    HMaster firstMaster = UTIL.getHBaseCluster().getMaster();
+    // Abort Latch for the master store
+    final CountDownLatch masterStoreAbort = new CountDownLatch(1);
+    masterStore.registerListener(new ProcedureStore.ProcedureStoreListener() {
+      @Override
+      public void postSync() {}
+
+      @Override
+      public void abortProcess() {
+        LOG.debug("Abort store of Master");
+        masterStoreAbort.countDown();
+      }
+    });
+
+    // startup a fake master the new WAL store will take the lease
+    // and the active master should abort.
+    HMaster backupMaster3 = Mockito.mock(HMaster.class);
+    Mockito.doReturn(firstMaster.getConfiguration()).when(backupMaster3).getConfiguration();
+    Mockito.doReturn(true).when(backupMaster3).isActiveMaster();
+    final WALProcedureStore backupStore3 = new WALProcedureStore(firstMaster.getConfiguration(),
+        firstMaster.getMasterFileSystem().getFileSystem(),
+        ((WALProcedureStore)masterStore).getLogDir(),
+        new MasterProcedureEnv.WALStoreLeaseRecovery(backupMaster3));
+    // Abort Latch for the test store
+    final CountDownLatch backupStore3Abort = new CountDownLatch(1);
+    backupStore3.registerListener(new ProcedureStore.ProcedureStoreListener() {
+      @Override
+      public void postSync() {}
+
+      @Override
+      public void abortProcess() {
+        LOG.debug("Abort store of backupMaster3");
+        backupStore3Abort.countDown();
+        backupStore3.stop(true);
+      }
+    });
+    backupStore3.start(1);
+    backupStore3.recoverLease();
+
+    // Try to trigger a command on the master (WAL lease expired on the active one)
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(TableName.valueOf("mtb"), "f");
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
+    LOG.debug("submit proc");
+    try {
+      getMasterProcedureExecutor().submitProcedure(
+          new CreateTableProcedure(getMasterProcedureExecutor().getEnvironment(), htd, regions));
+      fail("expected RuntimeException 'sync aborted'");
+    } catch (RuntimeException e) {
+      LOG.info("got " + e.getMessage());
+    }
+    LOG.debug("wait master store abort");
+    masterStoreAbort.await();
+
+    // Now the real backup master should start up
+    LOG.debug("wait backup master to startup");
+    waitBackupMaster(UTIL, firstMaster);
+    assertEquals(true, firstMaster.isStopped());
+
+    // wait the store in here to abort (the test will fail due to timeout if it doesn't)
+    LOG.debug("wait the store to abort");
+    backupStore3.getStoreTracker().setDeleted(1, false);
+    try {
+      backupStore3.delete(1);
+      fail("expected RuntimeException 'sync aborted'");
+    } catch (RuntimeException e) {
+      LOG.info("got " + e.getMessage());
+    }
+    backupStore3Abort.await();
+  }
+
+  /**
+   * Tests proper fencing in case the current WAL store is fenced
+   */
+  @Test
+  public void testWALfencingWithoutWALRolling() throws IOException {
+    testWALfencing(false);
+  }
+
+  /**
+   * Tests proper fencing in case the current WAL store does not receive writes until after the
+   * new WAL does a couple of WAL rolls.
+   */
+  @Test
+  public void testWALfencingWithWALRolling() throws IOException {
+    testWALfencing(true);
+  }
+
+  public void testWALfencing(boolean walRolls) throws IOException {
+    final ProcedureStore procStore = getMasterProcedureExecutor().getStore();
+    assertTrue("expected WALStore for this test", procStore instanceof WALProcedureStore);
+
+    HMaster firstMaster = UTIL.getHBaseCluster().getMaster();
+
+    // cause WAL rolling after a delete in WAL:
+    firstMaster.getConfiguration().setLong("hbase.procedure.store.wal.roll.threshold", 1);
+
+    HMaster backupMaster3 = Mockito.mock(HMaster.class);
+    Mockito.doReturn(firstMaster.getConfiguration()).when(backupMaster3).getConfiguration();
+    Mockito.doReturn(true).when(backupMaster3).isActiveMaster();
+    final WALProcedureStore procStore2 = new WALProcedureStore(firstMaster.getConfiguration(),
+        firstMaster.getMasterFileSystem().getFileSystem(),
+        ((WALProcedureStore)procStore).getLogDir(),
+        new MasterProcedureEnv.WALStoreLeaseRecovery(backupMaster3));
+
+    // start a second store which should fence the first one out
+    LOG.info("Starting new WALProcedureStore");
+    procStore2.start(1);
+    procStore2.recoverLease();
+
+    // before writing back to the WAL store, optionally do a couple of WAL rolls (which causes
+    // to delete the old WAL files).
+    if (walRolls) {
+      LOG.info("Inserting into second WALProcedureStore, causing WAL rolls");
+      for (int i = 0; i < 512; i++) {
+        // insert something to the second store then delete it, causing a WAL roll(s)
+        Procedure proc2 = new TestProcedure(i);
+        procStore2.insert(proc2, null);
+        procStore2.delete(proc2.getProcId()); // delete the procedure so that the WAL is removed later
+      }
+    }
+
+    // Now, insert something to the first store, should fail.
+    // If the store does a WAL roll and continue with another logId without checking higher logIds
+    // it will incorrectly succeed.
+    LOG.info("Inserting into first WALProcedureStore");
+    try {
+      procStore.insert(new TestProcedure(11), null);
+      fail("Inserting into Procedure Store should have failed");
+    } catch (Exception ex) {
+      LOG.info("Received expected exception", ex);
+    }
+  }
+
+  // ==========================================================================
+  //  Test Create Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testCreateWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestCreateTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testCreateWithFailoverAtStep(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS.ordinal());
+  }
+
+  private void testCreateWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testCreateWithFailoverAtStep" + step);
+
+    // create the table
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
+
+    // Start the Create procedure && kill the executor
+    byte[][] splitKeys = null;
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+    long procId = procExec.submitProcedure(
+        new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, CreateTableState.values());
+
+    MasterProcedureTestingUtility.validateTableCreation(
+        UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  // ==========================================================================
+  //  Test Delete Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testDeleteWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestDeleteTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testDeleteWithFailoverAtStep(DeleteTableState.DELETE_TABLE_UNASSIGN_REGIONS.ordinal());
+  }
+
+  private void testDeleteWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteWithFailoverAtStep" + step);
+
+    // create the table
+    byte[][] splitKeys = null;
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+        getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    Path tableDir = FSUtils.getTableDir(getRootDir(), tableName);
+    MasterProcedureTestingUtility.validateTableCreation(
+        UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+        new DeleteTableProcedure(procExec.getEnvironment(), tableName));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, DeleteTableState.values());
+
+    MasterProcedureTestingUtility.validateTableDeletion(
+        UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  // ==========================================================================
+  //  Test Truncate Table
+  // ==========================================================================
+  @Test(timeout=90000)
+  public void testTruncateWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestTruncateTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testTruncateWithFailoverAtStep(true, TruncateTableState.TRUNCATE_TABLE_ADD_TO_META.ordinal());
+  }
+
+  private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final int step)
+      throws Exception {
+    final TableName tableName = TableName.valueOf("testTruncateWithFailoverAtStep" + step);
+
+    // create the table
+    final String[] families = new String[] { "f1", "f2" };
+    final byte[][] splitKeys = new byte[][] {
+        Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+        getMasterProcedureExecutor(), tableName, splitKeys, families);
+    // load and verify that there are rows in the table
+    MasterProcedureTestingUtility.loadData(
+        UTIL.getConnection(), tableName, 100, splitKeys, families);
+    assertEquals(100, UTIL.countRows(tableName));
+    // disable the table
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Truncate procedure && kill the executor
+    long procId = procExec.submitProcedure(
+        new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, TruncateTableState.values());
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    UTIL.waitUntilAllRegionsAssigned(tableName);
+
+    // validate the table regions and layout
+    if (preserveSplits) {
+      assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
+    } else {
+      regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
+      assertEquals(1, regions.length);
+    }
+    MasterProcedureTestingUtility.validateTableCreation(
+        UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
+
+    // verify that there are no rows in the table
+    assertEquals(0, UTIL.countRows(tableName));
+
+    // verify that the table is read/writable
+    MasterProcedureTestingUtility.loadData(
+        UTIL.getConnection(), tableName, 50, splitKeys, families);
+    assertEquals(50, UTIL.countRows(tableName));
+  }
+
+  // ==========================================================================
+  //  Test Disable Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testDisableTableWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestDisableTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testDisableTableWithFailoverAtStep(
+        DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE.ordinal());
+  }
+
+  private void testDisableTableWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testDisableTableWithFailoverAtStep" + step);
+
+    // create the table
+    final byte[][] splitKeys = new byte[][] {
+        Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    MasterProcedureTestingUtility.createTable(
+        getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+        new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, DisableTableState.values());
+
+    MasterProcedureTestingUtility.validateTableIsDisabled(
+        UTIL.getHBaseCluster().getMaster(), tableName);
+  }
+
+  // ==========================================================================
+  //  Test Enable Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testEnableTableWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestEnableTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testEnableTableWithFailoverAtStep(
+        EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE.ordinal());
+  }
+
+  private void testEnableTableWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testEnableTableWithFailoverAtStep" + step);
+
+    // create the table
+    final byte[][] splitKeys = new byte[][] {
+        Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    MasterProcedureTestingUtility.createTable(
+        getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+        new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, EnableTableState.values());
+
+    MasterProcedureTestingUtility.validateTableIsEnabled(
+        UTIL.getHBaseCluster().getMaster(), tableName);
+  }
+
+  // ==========================================================================
+  //  Test Helpers
+  // ==========================================================================
+  public static <TState> void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
+      final long procId, final int lastStepBeforeFailover, TState[] states) throws Exception {
+    ProcedureExecutor<MasterProcedureEnv> procExec =
+        testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+
+    for (int i = 0; i < lastStepBeforeFailover; ++i) {
+      LOG.info("Restart "+ i +" exec state: " + states[i]);
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    }
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+
+    LOG.info("Trigger master failover");
+    masterFailover(testUtil);
+
+    procExec = testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+  }
+
+  // ==========================================================================
+  //  Master failover utils
+  // ==========================================================================
+  public static void masterFailover(final HBaseTestingUtility testUtil)
+      throws Exception {
+    MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
+
+    // Kill the master
+    HMaster oldMaster = cluster.getMaster();
+    cluster.killMaster(cluster.getMaster().getServerName());
+
+    // Wait the secondary
+    waitBackupMaster(testUtil, oldMaster);
+  }
+
+  public static void waitBackupMaster(final HBaseTestingUtility testUtil,
+      final HMaster oldMaster) throws Exception {
+    MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
+
+    HMaster newMaster = cluster.getMaster();
+    while (newMaster == null || newMaster == oldMaster) {
+      Thread.sleep(250);
+      newMaster = cluster.getMaster();
+    }
+
+    while (!(newMaster.isActiveMaster() && newMaster.isInitialized())) {
+      Thread.sleep(250);
+    }
+  }
+
+  // ==========================================================================
+  //  Helpers
+  // ==========================================================================
+  private MasterProcedureEnv getMasterProcedureEnv() {
+    return getMasterProcedureExecutor().getEnvironment();
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+
+  private FileSystem getFileSystem() {
+    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+  }
+
+  private Path getRootDir() {
+    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+  }
+
+  private Path getTempDir() {
+    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getTempDir();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
new file mode 100644
index 0000000..fe297edc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
@@ -0,0 +1,72 @@
+/**
+ * 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.snapshot;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test creating/using/deleting snapshots from the client
+ * <p>
+ * This is an end-to-end test for the snapshot utility
+ *
+ * TODO This is essentially a clone of TestSnapshotFromClient.  This is worth refactoring this
+ * because there will be a few more flavors of snapshots that need to run these tests.
+ */
+@Category({ClientTests.class, LargeTests.class})
+public class TestMobFlushSnapshotFromClient extends TestFlushSnapshotFromClient {
+  private static final Log LOG = LogFactory.getLog(TestFlushSnapshotFromClient.class);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(3);
+  }
+
+  protected static void setupConf(Configuration conf) {
+    TestFlushSnapshotFromClient.setupConf(conf);
+    UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+  }
+
+  @Override
+  protected void createTable() throws Exception {
+    MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, 1, TEST_FAM);
+  }
+
+  @Override
+  protected void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
+      long expectedRows) throws IOException {
+    MobSnapshotTestingUtils.verifyMobRowCount(util, tableName, expectedRows);
+  }
+
+  @Override
+  protected int countRows(final Table table, final byte[]... families) throws IOException {
+    return MobSnapshotTestingUtils.countMobRows(table, families);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/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
new file mode 100644
index 0000000..67fc60a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -0,0 +1,1320 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.wal;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.wal.FaultySequenceFileLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
+import org.apache.hadoop.hbase.wal.WALSplitter.CorruptedLogFileException;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Testing {@link WAL} splitting code.
+ */
+@Category({RegionServerTests.class, LargeTests.class})
+public class TestWALSplit {
+  {
+    // Uncomment the following lines if more verbosity is needed for
+    // debugging (see HBASE-12285 for details).
+    //((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
+    //((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
+    //((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
+  }
+  private final static Log LOG = LogFactory.getLog(TestWALSplit.class);
+
+  private static Configuration conf;
+  private FileSystem fs;
+
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private Path HBASEDIR;
+  private Path WALDIR;
+  private Path OLDLOGDIR;
+  private Path CORRUPTDIR;
+  private Path TABLEDIR;
+
+  private static final int NUM_WRITERS = 10;
+  private static final int ENTRIES = 10; // entries per writer per region
+
+  private static final String FILENAME_BEING_SPLIT = "testfile";
+  private static final TableName TABLE_NAME =
+      TableName.valueOf("t1");
+  private static final byte[] FAMILY = "f1".getBytes();
+  private static final byte[] QUALIFIER = "q1".getBytes();
+  private static final byte[] VALUE = "v1".getBytes();
+  private static final String WAL_FILE_PREFIX = "wal.dat.";
+  private static List<String> REGIONS = new ArrayList<String>();
+  private static final String HBASE_SKIP_ERRORS = "hbase.hlog.split.skip.errors";
+  private static String ROBBER;
+  private static String ZOMBIE;
+  private static String [] GROUP = new String [] {"supergroup"};
+  private RecoveryMode mode;
+
+  static enum Corruptions {
+    INSERT_GARBAGE_ON_FIRST_LINE,
+    INSERT_GARBAGE_IN_THE_MIDDLE,
+    APPEND_GARBAGE,
+    TRUNCATE,
+    TRUNCATE_TRAILER
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    conf.setClass("hbase.regionserver.hlog.writer.impl",
+        InstrumentedLogWriter.class, Writer.class);
+    conf.setBoolean("dfs.support.broken.append", true);
+    conf.setBoolean("dfs.support.append", true);
+    // This is how you turn off shortcircuit read currently.  TODO: Fix.  Should read config.
+    System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
+    // Create fake maping user to group and set it to the conf.
+    Map<String, String []> u2g_map = new HashMap<String, String []>(2);
+    ROBBER = User.getCurrent().getName() + "-robber";
+    ZOMBIE = User.getCurrent().getName() + "-zombie";
+    u2g_map.put(ROBBER, GROUP);
+    u2g_map.put(ZOMBIE, GROUP);
+    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
+    conf.setInt("dfs.heartbeat.interval", 1);
+    TEST_UTIL.startMiniDFSCluster(2);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniDFSCluster();
+  }
+
+  @Rule
+  public TestName name = new TestName();
+  private WALFactory wals = null;
+
+  @Before
+  public void setUp() throws Exception {
+    LOG.info("Cleaning up cluster for new test.");
+    fs = TEST_UTIL.getDFSCluster().getFileSystem();
+    HBASEDIR = TEST_UTIL.createRootDir();
+    OLDLOGDIR = new Path(HBASEDIR, HConstants.HREGION_OLDLOGDIR_NAME);
+    CORRUPTDIR = new Path(HBASEDIR, HConstants.CORRUPT_DIR_NAME);
+    TABLEDIR = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
+    REGIONS.clear();
+    Collections.addAll(REGIONS, "bbb", "ccc");
+    InstrumentedLogWriter.activateFailure = false;
+    this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
+        RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
+    wals = new WALFactory(conf, null, name.getMethodName());
+    WALDIR = new Path(HBASEDIR, DefaultWALProvider.getWALDirectoryName(name.getMethodName()));
+    //fs.mkdirs(WALDIR);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      wals.close();
+    } catch(IOException exception) {
+      // Some tests will move WALs out from under us. In those cases, we'll get an error on close.
+      LOG.info("Ignoring an error while closing down our WALFactory. Fine for some tests, but if" +
+          " you see a failure look here.");
+      LOG.debug("exception details", exception);
+    } finally {
+      wals = null;
+      fs.delete(HBASEDIR, true);
+    }
+  }
+
+  /**
+   * Simulates splitting a WAL out from under a regionserver that is still trying to write it.
+   * Ensures we do not lose edits.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test (timeout=300000)
+  public void testLogCannotBeWrittenOnceParsed() throws IOException, InterruptedException {
+    final AtomicLong counter = new AtomicLong(0);
+    AtomicBoolean stop = new AtomicBoolean(false);
+    // Region we'll write edits too and then later examine to make sure they all made it in.
+    final String region = REGIONS.get(0);
+    final int numWriters = 3;
+    Thread zombie = new ZombieLastLogWriterRegionServer(counter, stop, region, numWriters);
+    try {
+      long startCount = counter.get();
+      zombie.start();
+      // Wait till writer starts going.
+      while (startCount == counter.get()) Threads.sleep(1);
+      // Give it a second to write a few appends.
+      Threads.sleep(1000);
+      final Configuration conf2 = HBaseConfiguration.create(this.conf);
+      final User robber = User.createUserForTesting(conf2, ROBBER, GROUP);
+      int count = robber.runAs(new PrivilegedExceptionAction<Integer>() {
+        @Override
+        public Integer run() throws Exception {
+          StringBuilder ls = new StringBuilder("Contents of WALDIR (").append(WALDIR)
+              .append("):\n");
+          for (FileStatus status : fs.listStatus(WALDIR)) {
+            ls.append("\t").append(status.toString()).append("\n");
+          }
+          LOG.debug(ls);
+          LOG.info("Splitting WALs out from under zombie. Expecting " + numWriters + " files.");
+          WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf2, wals);
+          LOG.info("Finished splitting out from under zombie.");
+          Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region);
+          assertEquals("wrong number of split files for region", numWriters, logfiles.length);
+          int count = 0;
+          for (Path logfile: logfiles) {
+            count += countWAL(logfile);
+          }
+          return count;
+        }
+      });
+      LOG.info("zombie=" + counter.get() + ", robber=" + count);
+      assertTrue("The log file could have at most 1 extra log entry, but can't have less. " +
+              "Zombie could write " + counter.get() + " and logfile had only " + count,
+          counter.get() == count || counter.get() + 1 == count);
+    } finally {
+      stop.set(true);
+      zombie.interrupt();
+      Threads.threadDumpingIsAlive(zombie);
+    }
+  }
+
+  /**
+   * This thread will keep writing to a 'wal' file even after the split process has started.
+   * It simulates a region server that was considered dead but woke up and wrote some more to the
+   * last log entry. Does its writing as an alternate user in another filesystem instance to
+   * simulate better it being a regionserver.
+   */
+  class ZombieLastLogWriterRegionServer extends Thread {
+    final AtomicLong editsCount;
+    final AtomicBoolean stop;
+    final int numOfWriters;
+    /**
+     * Region to write edits for.
+     */
+    final String region;
+    final User user;
+
+    public ZombieLastLogWriterRegionServer(AtomicLong counter, AtomicBoolean stop,
+        final String region, final int writers)
+        throws IOException, InterruptedException {
+      super("ZombieLastLogWriterRegionServer");
+      setDaemon(true);
+      this.stop = stop;
+      this.editsCount = counter;
+      this.region = region;
+      this.user = User.createUserForTesting(conf, ZOMBIE, GROUP);
+      numOfWriters = writers;
+    }
+
+    @Override
+    public void run() {
+      try {
+        doWriting();
+      } catch (IOException e) {
+        LOG.warn(getName() + " Writer exiting " + e);
+      } catch (InterruptedException e) {
+        LOG.warn(getName() + " Writer exiting " + e);
+      }
+    }
+
+    private void doWriting() throws IOException, InterruptedException {
+      this.user.runAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          // Index of the WAL we want to keep open.  generateWALs will leave open the WAL whose
+          // index we supply here.
+          int walToKeepOpen = numOfWriters - 1;
+          // The below method writes numOfWriters files each with ENTRIES entries for a total of
+          // numOfWriters * ENTRIES added per column family in the region.
+          Writer writer = null;
+          try {
+            writer = generateWALs(numOfWriters, ENTRIES, walToKeepOpen);
+          } catch (IOException e1) {
+            throw new RuntimeException("Failed", e1);
+          }
+          // Update counter so has all edits written so far.
+          editsCount.addAndGet(numOfWriters * ENTRIES);
+          loop(writer);
+          // If we've been interruped, then things should have shifted out from under us.
+          // closing should error
+          try {
+            writer.close();
+            fail("Writing closing after parsing should give an error.");
+          } catch (IOException exception) {
+            LOG.debug("ignoring error when closing final writer.", exception);
+          }
+          return null;
+        }
+      });
+    }
+
+    private void loop(final Writer writer) {
+      byte [] regionBytes = Bytes.toBytes(this.region);
+      while (!stop.get()) {
+        try {
+          long seq = appendEntry(writer, TABLE_NAME, regionBytes,
+              ("r" + editsCount.get()).getBytes(), regionBytes, QUALIFIER, VALUE, 0);
+          long count = editsCount.incrementAndGet();
+          LOG.info(getName() + " sync count=" + count + ", seq=" + seq);
+          try {
+            Thread.sleep(1);
+          } catch (InterruptedException e) {
+            //
+          }
+        } catch (IOException ex) {
+          LOG.error(getName() + " ex " + ex.toString());
+          if (ex instanceof RemoteException) {
+            LOG.error("Juliet: got RemoteException " + ex.getMessage() +
+                " while writing " + (editsCount.get() + 1));
+          } else {
+            LOG.error(getName() + " failed to write....at " + editsCount.get());
+            fail("Failed to write " + editsCount.get());
+          }
+          break;
+        } catch (Throwable t) {
+          LOG.error(getName() + " HOW? " + t);
+          LOG.debug("exception details", t);
+          break;
+        }
+      }
+      LOG.info(getName() + " Writer exiting");
+    }
+  }
+
+  /**
+   * @throws IOException
+   * @see https://issues.apache.org/jira/browse/HBASE-3020
+   */
+  @Test (timeout=300000)
+  public void testRecoveredEditsPathForMeta() throws IOException {
+    byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
+    Path regiondir = new Path(tdir,
+        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    fs.mkdirs(regiondir);
+    long now = System.currentTimeMillis();
+    Entry entry =
+        new Entry(new WALKey(encoded,
+            TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
+            new WALEdit());
+    Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
+        FILENAME_BEING_SPLIT);
+    String parentOfParent = p.getParent().getParent().getName();
+    assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+  }
+
+  /**
+   * Test old recovered edits file doesn't break WALSplitter.
+   * This is useful in upgrading old instances.
+   */
+  @Test (timeout=300000)
+  public void testOldRecoveredEditsFileSidelined() throws IOException {
+    byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
+    Path regiondir = new Path(tdir,
+        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    fs.mkdirs(regiondir);
+    long now = System.currentTimeMillis();
+    Entry entry =
+        new Entry(new WALKey(encoded,
+            TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
+            new WALEdit());
+    Path parent = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
+    assertEquals(parent.getName(), HConstants.RECOVERED_EDITS_DIR);
+    fs.createNewFile(parent); // create a recovered.edits file
+
+    Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
+        FILENAME_BEING_SPLIT);
+    String parentOfParent = p.getParent().getParent().getName();
+    assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    WALFactory.createRecoveredEditsWriter(fs, p, conf).close();
+  }
+
+  private void useDifferentDFSClient() throws IOException {
+    // make fs act as a different client now
+    // initialize will create a new DFSClient with a new client ID
+    fs.initialize(fs.getUri(), conf);
+  }
+
+  @Test (timeout=300000)
+  public void testSplitPreservesEdits() throws IOException{
+    final String REGION = "region__1";
+    REGIONS.clear();
+    REGIONS.add(REGION);
+
+    generateWALs(1, 10, -1);
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    Path originalLog = (fs.listStatus(OLDLOGDIR))[0].getPath();
+    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    assertEquals(1, splitLog.length);
+
+    assertTrue("edits differ after split", logsAreEqual(originalLog, splitLog[0]));
+  }
+
+  /**
+   * @param expectedEntries -1 to not assert
+   * @return the count across all regions
+   */
+  private int splitAndCount(final int expectedFiles, final int expectedEntries)
+      throws IOException {
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    int result = 0;
+    for (String region : REGIONS) {
+      Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region);
+      assertEquals(expectedFiles, logfiles.length);
+      int count = 0;
+      for (Path logfile: logfiles) {
+        count += countWAL(logfile);
+      }
+      if (-1 != expectedEntries) {
+        assertEquals(expectedEntries, count);
+      }
+      result += count;
+    }
+    return result;
+  }
+
+  @Test (timeout=300000)
+  public void testEmptyLogFiles() throws IOException {
+    testEmptyLogFiles(true);
+  }
+
+  @Test (timeout=300000)
+  public void testEmptyOpenLogFiles() throws IOException {
+    testEmptyLogFiles(false);
+  }
+
+  private void testEmptyLogFiles(final boolean close) throws IOException {
+    // we won't create the hlog dir until getWAL got called, so
+    // make dir here when testing empty log file
+    fs.mkdirs(WALDIR);
+    injectEmptyFile(".empty", close);
+    generateWALs(Integer.MAX_VALUE);
+    injectEmptyFile("empty", close);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES); // skip 2 empty
+  }
+
+  @Test (timeout=300000)
+  public void testOpenZeroLengthReportedFileButWithDataGetsSplit() throws IOException {
+    // generate logs but leave wal.dat.5 open.
+    generateWALs(5);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES);
+  }
+
+  @Test (timeout=300000)
+  public void testTralingGarbageCorruptionFileSkipErrorsPasses() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(Integer.MAX_VALUE);
+    corruptWAL(new Path(WALDIR, WAL_FILE_PREFIX + "5"),
+        Corruptions.APPEND_GARBAGE, true);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES);
+  }
+
+  @Test (timeout=300000)
+  public void testFirstLineCorruptionLogFileSkipErrorsPasses() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(Integer.MAX_VALUE);
+    corruptWAL(new Path(WALDIR, WAL_FILE_PREFIX + "5"),
+        Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true);
+    splitAndCount(NUM_WRITERS - 1, (NUM_WRITERS - 1) * ENTRIES); //1 corrupt
+  }
+
+  @Test (timeout=300000)
+  public void testMiddleGarbageCorruptionSkipErrorsReadsHalfOfFile() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(Integer.MAX_VALUE);
+    corruptWAL(new Path(WALDIR, WAL_FILE_PREFIX + "5"),
+        Corruptions.INSERT_GARBAGE_IN_THE_MIDDLE, false);
+    // the entries in the original logs are alternating regions
+    // considering the sequence file header, the middle corruption should
+    // affect at least half of the entries
+    int goodEntries = (NUM_WRITERS - 1) * ENTRIES;
+    int firstHalfEntries = (int) Math.ceil(ENTRIES / 2) - 1;
+    int allRegionsCount = splitAndCount(NUM_WRITERS, -1);
+    assertTrue("The file up to the corrupted area hasn't been parsed",
+        REGIONS.size() * (goodEntries + firstHalfEntries) <= allRegionsCount);
+  }
+
+  @Test (timeout=300000)
+  public void testCorruptedFileGetsArchivedIfSkipErrors() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    for (FaultySequenceFileLogReader.FailureType  failureType :
+        FaultySequenceFileLogReader.FailureType.values()) {
+      final Set<String> walDirContents = splitCorruptWALs(failureType);
+      final Set<String> archivedLogs = new HashSet<String>();
+      final StringBuilder archived = new StringBuilder("Archived logs in CORRUPTDIR:");
+      for (FileStatus log : fs.listStatus(CORRUPTDIR)) {
+        archived.append("\n\t").append(log.toString());
+        archivedLogs.add(log.getPath().getName());
+      }
+      LOG.debug(archived.toString());
+      assertEquals(failureType.name() + ": expected to find all of our wals corrupt.",
+          walDirContents, archivedLogs);
+    }
+  }
+
+  /**
+   * @return set of wal names present prior to split attempt.
+   * @throws IOException if the split process fails
+   */
+  private Set<String> splitCorruptWALs(final FaultySequenceFileLogReader.FailureType failureType)
+      throws IOException {
+    Class<?> backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
+        Reader.class);
+    InstrumentedLogWriter.activateFailure = false;
+
+    try {
+      conf.setClass("hbase.regionserver.hlog.reader.impl",
+          FaultySequenceFileLogReader.class, Reader.class);
+      conf.set("faultysequencefilelogreader.failuretype", failureType.name());
+      // Clean up from previous tests or previous loop
+      try {
+        wals.shutdown();
+      } catch (IOException exception) {
+        // since we're splitting out from under the factory, we should expect some closing failures.
+        LOG.debug("Ignoring problem closing WALFactory.", exception);
+      }
+      wals.close();
+      try {
+        for (FileStatus log : fs.listStatus(CORRUPTDIR)) {
+          fs.delete(log.getPath(), true);
+        }
+      } catch (FileNotFoundException exception) {
+        LOG.debug("no previous CORRUPTDIR to clean.");
+      }
+      // change to the faulty reader
+      wals = new WALFactory(conf, null, name.getMethodName());
+      generateWALs(-1);
+      // Our reader will render all of these files corrupt.
+      final Set<String> walDirContents = new HashSet<String>();
+      for (FileStatus status : fs.listStatus(WALDIR)) {
+        walDirContents.add(status.getPath().getName());
+      }
+      useDifferentDFSClient();
+      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+      return walDirContents;
+    } finally {
+      conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass,
+          Reader.class);
+    }
+  }
+
+  @Test (timeout=300000, expected = IOException.class)
+  public void testTrailingGarbageCorruptionLogFileSkipErrorsFalseThrows()
+      throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+    splitCorruptWALs(FaultySequenceFileLogReader.FailureType.BEGINNING);
+  }
+
+  @Test (timeout=300000)
+  public void testCorruptedLogFilesSkipErrorsFalseDoesNotTouchLogs()
+      throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+    try {
+      splitCorruptWALs(FaultySequenceFileLogReader.FailureType.BEGINNING);
+    } catch (IOException e) {
+      LOG.debug("split with 'skip errors' set to 'false' correctly threw");
+    }
+    assertEquals("if skip.errors is false all files should remain in place",
+        NUM_WRITERS, fs.listStatus(WALDIR).length);
+  }
+
+  private void ignoreCorruption(final Corruptions corruption, final int entryCount,
+      final int expectedCount) throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+
+    final String REGION = "region__1";
+    REGIONS.clear();
+    REGIONS.add(REGION);
+
+    Path c1 = new Path(WALDIR, WAL_FILE_PREFIX + "0");
+    generateWALs(1, entryCount, -1);
+    corruptWAL(c1, corruption, true);
+
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+
+    Path[] splitLog = getLogForRegion(HBASEDIR, TABLE_NAME, REGION);
+    assertEquals(1, splitLog.length);
+
+    int actualCount = 0;
+    Reader in = wals.createReader(fs, splitLog[0]);
+    @SuppressWarnings("unused")
+    Entry entry;
+    while ((entry = in.next()) != null) ++actualCount;
+    assertEquals(expectedCount, actualCount);
+    in.close();
+
+    // should not have stored the EOF files as corrupt
+    FileStatus[] archivedLogs = fs.listStatus(CORRUPTDIR);
+    assertEquals(archivedLogs.length, 0);
+
+  }
+
+  @Test (timeout=300000)
+  public void testEOFisIgnored() throws IOException {
+    int entryCount = 10;
+    ignoreCorruption(Corruptions.TRUNCATE, entryCount, entryCount-1);
+  }
+
+  @Test (timeout=300000)
+  public void testCorruptWALTrailer() throws IOException {
+    int entryCount = 10;
+    ignoreCorruption(Corruptions.TRUNCATE_TRAILER, entryCount, entryCount);
+  }
+
+  @Test (timeout=300000)
+  public void testLogsGetArchivedAfterSplit() throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+    generateWALs(-1);
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    FileStatus[] archivedLogs = fs.listStatus(OLDLOGDIR);
+    assertEquals("wrong number of files in the archive log", NUM_WRITERS, archivedLogs.length);
+  }
+
+  @Test (timeout=300000)
+  public void testSplit() throws IOException {
+    generateWALs(-1);
+    splitAndCount(NUM_WRITERS, NUM_WRITERS * ENTRIES);
+  }
+
+  @Test (timeout=300000)
+  public void testLogDirectoryShouldBeDeletedAfterSuccessfulSplit()
+      throws IOException {
+    generateWALs(-1);
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    FileStatus [] statuses = null;
+    try {
+      statuses = fs.listStatus(WALDIR);
+      if (statuses != null) {
+        fail("Files left in log dir: " +
+            Joiner.on(",").join(FileUtil.stat2Paths(statuses)));
+      }
+    } catch (FileNotFoundException e) {
+      // hadoop 0.21 throws FNFE whereas hadoop 0.20 returns null
+    }
+  }
+
+  @Test(timeout=300000, expected = IOException.class)
+  public void testSplitWillFailIfWritingToRegionFails() throws Exception {
+    //leave 5th log open so we could append the "trap"
+    Writer writer = generateWALs(4);
+    useDifferentDFSClient();
+
+    String region = "break";
+    Path regiondir = new Path(TABLEDIR, region);
+    fs.mkdirs(regiondir);
+
+    InstrumentedLogWriter.activateFailure = false;
+    appendEntry(writer, TABLE_NAME, Bytes.toBytes(region),
+        ("r" + 999).getBytes(), FAMILY, QUALIFIER, VALUE, 0);
+    writer.close();
+
+    try {
+      InstrumentedLogWriter.activateFailure = true;
+      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    } catch (IOException e) {
+      assertTrue(e.getMessage().
+          contains("This exception is instrumented and should only be thrown for testing"));
+      throw e;
+    } finally {
+      InstrumentedLogWriter.activateFailure = false;
+    }
+  }
+
+  @Test (timeout=300000)
+  public void testSplitDeletedRegion() throws IOException {
+    REGIONS.clear();
+    String region = "region_that_splits";
+    REGIONS.add(region);
+
+    generateWALs(1);
+    useDifferentDFSClient();
+
+    Path regiondir = new Path(TABLEDIR, region);
+    fs.delete(regiondir, true);
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    assertFalse(fs.exists(regiondir));
+  }
+
+  @Test (timeout=300000)
+  public void testIOEOnOutputThread() throws Exception {
+    conf.setBoolean(HBASE_SKIP_ERRORS, false);
+
+    generateWALs(-1);
+    useDifferentDFSClient();
+    FileStatus[] logfiles = fs.listStatus(WALDIR);
+    assertTrue("There should be some log file",
+        logfiles != null && logfiles.length > 0);
+    // wals with no entries (like the one we don't use in the factory)
+    // won't cause a failure since nothing will ever be written.
+    // pick the largest one since it's most likely to have entries.
+    int largestLogFile = 0;
+    long largestSize = 0;
+    for (int i = 0; i < logfiles.length; i++) {
+      if (logfiles[i].getLen() > largestSize) {
+        largestLogFile = i;
+        largestSize = logfiles[i].getLen();
+      }
+    }
+    assertTrue("There should be some log greater than size 0.", 0 < largestSize);
+    // Set up a splitter that will throw an IOE on the output side
+    WALSplitter logSplitter = new WALSplitter(wals,
+        conf, HBASEDIR, fs, null, null, this.mode) {
+      @Override
+      protected Writer createWriter(Path logfile) throws IOException {
+        Writer mockWriter = Mockito.mock(Writer.class);
+        Mockito.doThrow(new IOException("Injected")).when(
+            mockWriter).append(Mockito.<Entry>any());
+        return mockWriter;
+      }
+    };
+    // Set up a background thread dumper.  Needs a thread to depend on and then we need to run
+    // the thread dumping in a background thread so it does not hold up the test.
+    final AtomicBoolean stop = new AtomicBoolean(false);
+    final Thread someOldThread = new Thread("Some-old-thread") {
+      @Override
+      public void run() {
+        while(!stop.get()) Threads.sleep(10);
+      }
+    };
+    someOldThread.setDaemon(true);
+    someOldThread.start();
+    final Thread t = new Thread("Background-thread-dumper") {
+      public void run() {
+        try {
+          Threads.threadDumpingIsAlive(someOldThread);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+    };
+    t.setDaemon(true);
+    t.start();
+    try {
+      logSplitter.splitLogFile(logfiles[largestLogFile], null);
+      fail("Didn't throw!");
+    } catch (IOException ioe) {
+      assertTrue(ioe.toString().contains("Injected"));
+    } finally {
+      // Setting this to true will turn off the background thread dumper.
+      stop.set(true);
+    }
+  }
+
+  /**
+   * @param spiedFs should be instrumented for failure.
+   */
+  private void retryOverHdfsProblem(final FileSystem spiedFs) throws Exception {
+    generateWALs(-1);
+    useDifferentDFSClient();
+
+    try {
+      WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, spiedFs, conf, wals);
+      assertEquals(NUM_WRITERS, fs.listStatus(OLDLOGDIR).length);
+      assertFalse(fs.exists(WALDIR));
+    } catch (IOException e) {
+      fail("There shouldn't be any exception but: " + e.toString());
+    }
+  }
+
+  // Test for HBASE-3412
+  @Test (timeout=300000)
+  public void testMovedWALDuringRecovery() throws Exception {
+    // This partial mock will throw LEE for every file simulating
+    // files that were moved
+    FileSystem spiedFs = Mockito.spy(fs);
+    // The "File does not exist" part is very important,
+    // that's how it comes out of HDFS
+    Mockito.doThrow(new LeaseExpiredException("Injected: File does not exist")).
+        when(spiedFs).append(Mockito.<Path>any());
+    retryOverHdfsProblem(spiedFs);
+  }
+
+  @Test (timeout=300000)
+  public void testRetryOpenDuringRecovery() throws Exception {
+    FileSystem spiedFs = Mockito.spy(fs);
+    // The "Cannot obtain block length", "Could not obtain the last block",
+    // and "Blocklist for [^ ]* has changed.*" part is very important,
+    // that's how it comes out of HDFS. If HDFS changes the exception
+    // message, this test needs to be adjusted accordingly.
+    //
+    // When DFSClient tries to open a file, HDFS needs to locate
+    // the last block of the file and get its length. However, if the
+    // last block is under recovery, HDFS may have problem to obtain
+    // the block length, in which case, retry may help.
+    Mockito.doAnswer(new Answer<FSDataInputStream>() {
+      private final String[] errors = new String[] {
+          "Cannot obtain block length", "Could not obtain the last block",
+          "Blocklist for " + OLDLOGDIR + " has changed"};
+      private int count = 0;
+
+      public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable {
+        if (count < 3) {
+          throw new IOException(errors[count++]);
+        }
+        return (FSDataInputStream)invocation.callRealMethod();
+      }
+    }).when(spiedFs).open(Mockito.<Path>any(), Mockito.anyInt());
+    retryOverHdfsProblem(spiedFs);
+  }
+
+  @Test (timeout=300000)
+  public void testTerminationAskedByReporter() throws IOException, CorruptedLogFileException {
+    generateWALs(1, 10, -1);
+    FileStatus logfile = fs.listStatus(WALDIR)[0];
+    useDifferentDFSClient();
+
+    final AtomicInteger count = new AtomicInteger();
+
+    CancelableProgressable localReporter
+        = new CancelableProgressable() {
+      @Override
+      public boolean progress() {
+        count.getAndIncrement();
+        return false;
+      }
+    };
+
+    FileSystem spiedFs = Mockito.spy(fs);
+    Mockito.doAnswer(new Answer<FSDataInputStream>() {
+      public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable {
+        Thread.sleep(1500); // Sleep a while and wait report status invoked
+        return (FSDataInputStream)invocation.callRealMethod();
+      }
+    }).when(spiedFs).open(Mockito.<Path>any(), Mockito.anyInt());
+
+    try {
+      conf.setInt("hbase.splitlog.report.period", 1000);
+      boolean ret = WALSplitter.splitLogFile(
+          HBASEDIR, logfile, spiedFs, conf, localReporter, null, null, this.mode, wals);
+      assertFalse("Log splitting should failed", ret);
+      assertTrue(count.get() > 0);
+    } catch (IOException e) {
+      fail("There shouldn't be any exception but: " + e.toString());
+    } finally {
+      // reset it back to its default value
+      conf.setInt("hbase.splitlog.report.period", 59000);
+    }
+  }
+
+  /**
+   * Test log split process with fake data and lots of edits to trigger threading
+   * issues.
+   */
+  @Test (timeout=300000)
+  public void testThreading() throws Exception {
+    doTestThreading(20000, 128*1024*1024, 0);
+  }
+
+  /**
+   * Test blocking behavior of the log split process if writers are writing slower
+   * than the reader is reading.
+   */
+  @Test (timeout=300000)
+  public void testThreadingSlowWriterSmallBuffer() throws Exception {
+    doTestThreading(200, 1024, 50);
+  }
+
+  /**
+   * Sets up a log splitter with a mock reader and writer. The mock reader generates
+   * a specified number of edits spread across 5 regions. The mock writer optionally
+   * sleeps for each edit it is fed.
+   * *
+   * After the split is complete, verifies that the statistics show the correct number
+   * of edits output into each region.
+   *
+   * @param numFakeEdits number of fake edits to push through pipeline
+   * @param bufferSize size of in-memory buffer
+   * @param writerSlowness writer threads will sleep this many ms per edit
+   */
+  private void doTestThreading(final int numFakeEdits,
+      final int bufferSize,
+      final int writerSlowness) throws Exception {
+
+    Configuration localConf = new Configuration(conf);
+    localConf.setInt("hbase.regionserver.hlog.splitlog.buffersize", bufferSize);
+
+    // Create a fake log file (we'll override the reader to produce a stream of edits)
+    Path logPath = new Path(WALDIR, WAL_FILE_PREFIX + ".fake");
+    FSDataOutputStream out = fs.create(logPath);
+    out.close();
+
+    // Make region dirs for our destination regions so the output doesn't get skipped
+    final List<String> regions = ImmutableList.of("r0", "r1", "r2", "r3", "r4");
+    makeRegionDirs(regions);
+
+    // Create a splitter that reads and writes the data without touching disk
+    WALSplitter logSplitter = new WALSplitter(wals,
+        localConf, HBASEDIR, fs, null, null, this.mode) {
+
+      /* Produce a mock writer that doesn't write anywhere */
+      @Override
+      protected Writer createWriter(Path logfile) throws IOException {
+        Writer mockWriter = Mockito.mock(Writer.class);
+        Mockito.doAnswer(new Answer<Void>() {
+          int expectedIndex = 0;
+
+          @Override
+          public Void answer(InvocationOnMock invocation) {
+            if (writerSlowness > 0) {
+              try {
+                Thread.sleep(writerSlowness);
+              } catch (InterruptedException ie) {
+                Thread.currentThread().interrupt();
+              }
+            }
+            Entry entry = (Entry) invocation.getArguments()[0];
+            WALEdit edit = entry.getEdit();
+            List<Cell> cells = edit.getCells();
+            assertEquals(1, cells.size());
+            Cell cell = cells.get(0);
+
+            // Check that the edits come in the right order.
+            assertEquals(expectedIndex, Bytes.toInt(cell.getRowArray(), cell.getRowOffset(),
+                cell.getRowLength()));
+            expectedIndex++;
+            return null;
+          }
+        }).when(mockWriter).append(Mockito.<Entry>any());
+        return mockWriter;
+      }
+
+      /* Produce a mock reader that generates fake entries */
+      @Override
+      protected Reader getReader(Path curLogFile, CancelableProgressable reporter)
+          throws IOException {
+        Reader mockReader = Mockito.mock(Reader.class);
+        Mockito.doAnswer(new Answer<Entry>() {
+          int index = 0;
+
+          @Override
+          public Entry answer(InvocationOnMock invocation) throws Throwable {
+            if (index >= numFakeEdits) return null;
+
+            // Generate r0 through r4 in round robin fashion
+            int regionIdx = index % regions.size();
+            byte region[] = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
+
+            Entry ret = createTestEntry(TABLE_NAME, region,
+                Bytes.toBytes((int)(index / regions.size())),
+                FAMILY, QUALIFIER, VALUE, index);
+            index++;
+            return ret;
+          }
+        }).when(mockReader).next();
+        return mockReader;
+      }
+    };
+
+    logSplitter.splitLogFile(fs.getFileStatus(logPath), null);
+
+    // Verify number of written edits per region
+    Map<byte[], Long> outputCounts = logSplitter.outputSink.getOutputCounts();
+    for (Map.Entry<byte[], Long> entry : outputCounts.entrySet()) {
+      LOG.info("Got " + entry.getValue() + " output edits for region " +
+          Bytes.toString(entry.getKey()));
+      assertEquals((long)entry.getValue(), numFakeEdits / regions.size());
+    }
+    assertEquals("Should have as many outputs as regions", regions.size(), outputCounts.size());
+  }
+
+  // Does leaving the writer open in testSplitDeletedRegion matter enough for two tests?
+  @Test (timeout=300000)
+  public void testSplitLogFileDeletedRegionDir() throws IOException {
+    LOG.info("testSplitLogFileDeletedRegionDir");
+    final String REGION = "region__1";
+    REGIONS.clear();
+    REGIONS.add(REGION);
+
+    generateWALs(1, 10, -1);
+    useDifferentDFSClient();
+
+    Path regiondir = new Path(TABLEDIR, REGION);
+    LOG.info("Region directory is" + regiondir);
+    fs.delete(regiondir, true);
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    assertFalse(fs.exists(regiondir));
+  }
+
+  @Test (timeout=300000)
+  public void testSplitLogFileEmpty() throws IOException {
+    LOG.info("testSplitLogFileEmpty");
+    // we won't create the hlog dir until getWAL got called, so
+    // make dir here when testing empty log file
+    fs.mkdirs(WALDIR);
+    injectEmptyFile(".empty", true);
+    useDifferentDFSClient();
+
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+    Path tdir = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
+    assertFalse(fs.exists(tdir));
+
+    assertEquals(0, countWAL(fs.listStatus(OLDLOGDIR)[0].getPath()));
+  }
+
+  @Test (timeout=300000)
+  public void testSplitLogFileMultipleRegions() throws IOException {
+    LOG.info("testSplitLogFileMultipleRegions");
+    generateWALs(1, 10, -1);
+    splitAndCount(1, 10);
+  }
+
+  @Test (timeout=300000)
+  public void testSplitLogFileFirstLineCorruptionLog()
+      throws IOException {
+    conf.setBoolean(HBASE_SKIP_ERRORS, true);
+    generateWALs(1, 10, -1);
+    FileStatus logfile = fs.listStatus(WALDIR)[0];
+
+    corruptWAL(logfile.getPath(),
+        Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true);
+
+    useDifferentDFSClient();
+    WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf, wals);
+
+    final Path corruptDir = new Path(FSUtils.getRootDir(conf), conf.get(
+        "hbase.regionserver.hlog.splitlog.corrupt.dir", HConstants.CORRUPT_DIR_NAME));
+    assertEquals(1, fs.listStatus(corruptDir).length);
+  }
+
+  /**
+   * @throws IOException
+   * @see https://issues.apache.org/jira/browse/HBASE-4862
+   */
+  @Test (timeout=300000)
+  public void testConcurrentSplitLogAndReplayRecoverEdit() throws IOException {
+    LOG.info("testConcurrentSplitLogAndReplayRecoverEdit");
+    // Generate wals for our destination region
+    String regionName = "r0";
+    final Path regiondir = new Path(TABLEDIR, regionName);
+    REGIONS.clear();
+    REGIONS.add(regionName);
+    generateWALs(-1);
+
+    wals.getWAL(Bytes.toBytes(regionName), null);
+    FileStatus[] logfiles = fs.listStatus(WALDIR);
+    assertTrue("There should be some log file",
+        logfiles != null && logfiles.length > 0);
+
+    WALSplitter logSplitter = new WALSplitter(wals,
+        conf, HBASEDIR, fs, null, null, this.mode) {
+      @Override
+      protected Writer createWriter(Path logfile)
+          throws IOException {
+        Writer writer = wals.createRecoveredEditsWriter(this.fs, logfile);
+        // After creating writer, simulate region's
+        // replayRecoveredEditsIfAny() which gets SplitEditFiles of this
+        // region and delete them, excluding files with '.temp' suffix.
+        NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
+        if (files != null && !files.isEmpty()) {
+          for (Path file : files) {
+            if (!this.fs.delete(file, false)) {
+              LOG.error("Failed delete of " + file);
+            } else {
+              LOG.debug("Deleted recovered.edits file=" + file);
+            }
+          }
+        }
+        return writer;
+      }
+    };
+    try{
+      logSplitter.splitLogFile(logfiles[0], null);
+    } catch (IOException e) {
+      LOG.info(e);
+      fail("Throws IOException when spliting "
+          + "log, it is most likely because writing file does not "
+          + "exist which is caused by concurrent replayRecoveredEditsIfAny()");
+    }
+    if (fs.exists(CORRUPTDIR)) {
+      if (fs.listStatus(CORRUPTDIR).length > 0) {
+        fail("There are some corrupt logs, "
+            + "it is most likely caused by concurrent replayRecoveredEditsIfAny()");
+      }
+    }
+  }
+
+  private Writer generateWALs(int leaveOpen) throws IOException {
+    return generateWALs(NUM_WRITERS, ENTRIES, leaveOpen);
+  }
+
+  private void makeRegionDirs(List<String> regions) throws IOException {
+    for (String region : regions) {
+      LOG.debug("Creating dir for region " + region);
+      fs.mkdirs(new Path(TABLEDIR, region));
+    }
+  }
+
+  /**
+   * @param leaveOpen index to leave un-closed. -1 to close all.
+   * @return the writer that's still open, or null if all were closed.
+   */
+  private Writer generateWALs(int writers, int entries, int leaveOpen) throws IOException {
+    makeRegionDirs(REGIONS);
+    fs.mkdirs(WALDIR);
+    Writer [] ws = new Writer[writers];
+    int seq = 0;
+    for (int i = 0; i < writers; i++) {
+      ws[i] = wals.createWALWriter(fs, new Path(WALDIR, WAL_FILE_PREFIX + i));
+      for (int j = 0; j < entries; j++) {
+        int prefix = 0;
+        for (String region : REGIONS) {
+          String row_key = region + prefix++ + i + j;
+          appendEntry(ws[i], TABLE_NAME, region.getBytes(), row_key.getBytes(), FAMILY, QUALIFIER,
+              VALUE, seq++);
+        }
+      }
+      if (i != leaveOpen) {
+        ws[i].close();
+        LOG.info("Closing writer " + i);
+      }
+    }
+    if (leaveOpen < 0 || leaveOpen >= writers) {
+      return null;
+    }
+    return ws[leaveOpen];
+  }
+
+  private Path[] getLogForRegion(Path rootdir, TableName table, String region)
+      throws IOException {
+    Path tdir = FSUtils.getTableDir(rootdir, table);
+    @SuppressWarnings("deprecation")
+    Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
+        Bytes.toString(region.getBytes())));
+    FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
+      @Override
+      public boolean accept(Path p) {
+        if (WALSplitter.isSequenceIdFile(p)) {
+          return false;
+        }
+        return true;
+      }
+    });
+    Path[] paths = new Path[files.length];
+    for (int i = 0; i < files.length; i++) {
+      paths[i] = files[i].getPath();
+    }
+    return paths;
+  }
+
+  private void corruptWAL(Path path, Corruptions corruption, boolean close) throws IOException {
+    FSDataOutputStream out;
+    int fileSize = (int) fs.listStatus(path)[0].getLen();
+
+    FSDataInputStream in = fs.open(path);
+    byte[] corrupted_bytes = new byte[fileSize];
+    in.readFully(0, corrupted_bytes, 0, fileSize);
+    in.close();
+
+    switch (corruption) {
+    case APPEND_GARBAGE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(corrupted_bytes);
+      out.write("-----".getBytes());
+      closeOrFlush(close, out);
+      break;
+
+    case INSERT_GARBAGE_ON_FIRST_LINE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(0);
+      out.write(corrupted_bytes);
+      closeOrFlush(close, out);
+      break;
+
+    case INSERT_GARBAGE_IN_THE_MIDDLE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      int middle = (int) Math.floor(corrupted_bytes.length / 2);
+      out.write(corrupted_bytes, 0, middle);
+      out.write(0);
+      out.write(corrupted_bytes, middle, corrupted_bytes.length - middle);
+      closeOrFlush(close, out);
+      break;
+
+    case TRUNCATE:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(corrupted_bytes, 0, fileSize
+          - (32 + ProtobufLogReader.PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT));
+      closeOrFlush(close, out);
+      break;
+
+    case TRUNCATE_TRAILER:
+      fs.delete(path, false);
+      out = fs.create(path);
+      out.write(corrupted_bytes, 0, fileSize - Bytes.SIZEOF_INT);// trailer is truncated.
+      closeOrFlush(close, out);
+      break;
+    }
+  }
+
+  private void closeOrFlush(boolean close, FSDataOutputStream out)
+      throws IOException {
+    if (close) {
+      out.close();
+    } else {
+      Method syncMethod = null;
+      try {
+        syncMethod = out.getClass().getMethod("hflush", new Class<?> []{});
+      } catch (NoSuchMethodException e) {
+        try {
+          syncMethod = out.getClass().getMethod("sync", new Class<?> []{});
+        } catch (NoSuchMethodException ex) {
+          throw new IOException("This version of Hadoop supports " +
+              "neither Syncable.sync() nor Syncable.hflush().");
+        }
+      }
+      try {
+        syncMethod.invoke(out, new Object[]{});
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+      // Not in 0out.hflush();
+    }
+  }
+
+  private int countWAL(Path log) throws IOException {
+    int count = 0;
+    Reader in = wals.createReader(fs, log);
+    while (in.next() != null) {
+      count++;
+    }
+    in.close();
+    return count;
+  }
+
+  public static long appendEntry(Writer writer, TableName table, byte[] region,
+      byte[] row, byte[] family, byte[] qualifier,
+      byte[] value, long seq)
+      throws IOException {
+    LOG.info(Thread.currentThread().getName() + " append");
+    writer.append(createTestEntry(table, region, row, family, qualifier, value, seq));
+    LOG.info(Thread.currentThread().getName() + " sync");
+    writer.sync();
+    return seq;
+  }
+
+  private static Entry createTestEntry(
+      TableName table, byte[] region,
+      byte[] row, byte[] family, byte[] qualifier,
+      byte[] value, long seq) {
+    long time = System.nanoTime();
+    WALEdit edit = new WALEdit();
+    seq++;
+    edit.add(new KeyValue(row, family, qualifier, time, KeyValue.Type.Put, value));
+    return new Entry(new WALKey(region, table, seq, time,
+        HConstants.DEFAULT_CLUSTER_ID), edit);
+  }
+
+  private void injectEmptyFile(String suffix, boolean closeFile)
+      throws IOException {
+    Writer writer = wals.createWALWriter(fs, new Path(WALDIR, WAL_FILE_PREFIX + suffix),
+        conf);
+    if (closeFile) writer.close();
+  }
+
+  private boolean logsAreEqual(Path p1, Path p2) throws IOException {
+    Reader in1, in2;
+    in1 = wals.createReader(fs, p1);
+    in2 = wals.createReader(fs, p2);
+    Entry entry1;
+    Entry entry2;
+    while ((entry1 = in1.next()) != null) {
+      entry2 = in2.next();
+      if ((entry1.getKey().compareTo(entry2.getKey()) != 0) ||
+          (!entry1.getEdit().toString().equals(entry2.getEdit().toString()))) {
+        return false;
+      }
+    }
+    in1.close();
+    in2.close();
+    return true;
+  }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/30cec72f/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
new file mode 100644
index 0000000..3f4af05
--- /dev/null
+++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestReplicationShell.java
@@ -0,0 +1,37 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.jruby.embed.PathType;
+import org.junit.Test;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, LargeTests.class })
+public class TestReplicationShell extends AbstractTestShell {
+  @Ignore ("Disabled because hangs on occasion.. about 10% of the time") @Test
+  public void testRunShellTests() throws IOException {
+    System.setProperty("shell.test.include", "replication_admin_test.rb");
+    // Start all ruby tests
+    jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb");
+  }
+}
\ No newline at end of file


[07/37] hbase git commit: HBASE-15184 SparkSQL Scan operation doesn't work on kerberos cluster (Ted Malaska)

Posted by sy...@apache.org.
HBASE-15184 SparkSQL Scan operation doesn't work on kerberos cluster (Ted Malaska)


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

Branch: refs/heads/hbase-12439
Commit: 00248656ee9c60009ff1697e90ba9d0f86264103
Parents: f47dba7
Author: tedyu <yu...@gmail.com>
Authored: Tue Feb 23 16:52:13 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Feb 23 16:52:13 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/spark/DefaultSource.scala      |  4 +--
 .../hadoop/hbase/spark/HBaseContext.scala       | 15 +++++---
 .../apache/hadoop/hbase/spark/NewHBaseRDD.scala | 36 ++++++++++++++++++++
 .../spark/datasources/HBaseTableScanRDD.scala   | 15 ++++----
 4 files changed, 57 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/00248656/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
index b6d7982..844b5b5 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
@@ -164,7 +164,7 @@ case class HBaseRelation (val tableName:String,
     HBaseSparkConf.BULKGET_SIZE,  HBaseSparkConf.defaultBulkGetSize))
 
   //create or get latest HBaseContext
-  @transient val hbaseContext:HBaseContext = if (useHBaseContext) {
+  val hbaseContext:HBaseContext = if (useHBaseContext) {
     LatestHBaseContextCache.latest
   } else {
     val config = HBaseConfiguration.create()
@@ -270,7 +270,7 @@ case class HBaseRelation (val tableName:String,
     } else {
       None
     }
-    val hRdd = new HBaseTableScanRDD(this, pushDownFilterJava, requiredQualifierDefinitionList.seq)
+    val hRdd = new HBaseTableScanRDD(this, hbaseContext, pushDownFilterJava, requiredQualifierDefinitionList.seq)
     pushDownRowKeyFilter.points.foreach(hRdd.addPoint(_))
     pushDownRowKeyFilter.ranges.foreach(hRdd.addRange(_))
     var resultRDD: RDD[Row] = {

http://git-wip-us.apache.org/repos/asf/hbase/blob/00248656/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
index 2d21e69..61ed3cf 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
 import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFileContextBuilder, HFileWriterImpl}
 import org.apache.hadoop.hbase.regionserver.{HStore, StoreFile, BloomType}
 import org.apache.hadoop.hbase.util.Bytes
+import org.apache.hadoop.mapred.JobConf
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.rdd.RDD
@@ -228,7 +229,7 @@ class HBaseContext(@transient sc: SparkContext,
         }))
   }
 
-  def applyCreds[T] (configBroadcast: Broadcast[SerializableWritable[Configuration]]){
+  def applyCreds[T] (){
     credentials = SparkHadoopUtil.get.getCurrentUserCredentials()
 
     logDebug("appliedCredentials:" + appliedCredentials + ",credentials:" + credentials)
@@ -440,10 +441,14 @@ class HBaseContext(@transient sc: SparkContext,
     TableMapReduceUtil.initTableMapperJob(tableName, scan,
       classOf[IdentityTableMapper], null, null, job)
 
-    sc.newAPIHadoopRDD(job.getConfiguration,
+    val jconf = new JobConf(job.getConfiguration)
+    SparkHadoopUtil.get.addCredentials(jconf)
+    new NewHBaseRDD(sc,
       classOf[TableInputFormat],
       classOf[ImmutableBytesWritable],
-      classOf[Result]).map(f)
+      classOf[Result],
+      job.getConfiguration,
+      this).map(f)
   }
 
   /**
@@ -474,7 +479,7 @@ class HBaseContext(@transient sc: SparkContext,
 
     val config = getConf(configBroadcast)
 
-    applyCreds(configBroadcast)
+    applyCreds
     // specify that this is a proxy user
     val connection = ConnectionFactory.createConnection(config)
     f(it, connection)
@@ -514,7 +519,7 @@ class HBaseContext(@transient sc: SparkContext,
                                          Iterator[U]): Iterator[U] = {
 
     val config = getConf(configBroadcast)
-    applyCreds(configBroadcast)
+    applyCreds
 
     val connection = ConnectionFactory.createConnection(config)
     val res = mp(it, connection)

http://git-wip-us.apache.org/repos/asf/hbase/blob/00248656/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala
new file mode 100644
index 0000000..8e5e8f9
--- /dev/null
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.spark
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.InputFormat
+import org.apache.spark.rdd.NewHadoopRDD
+import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext}
+
+class NewHBaseRDD[K,V](@transient sc : SparkContext,
+                       @transient inputFormatClass: Class[_ <: InputFormat[K, V]],
+                       @transient keyClass: Class[K],
+                       @transient valueClass: Class[V],
+                   @transient conf: Configuration,
+                   val hBaseContext: HBaseContext) extends NewHadoopRDD(sc,inputFormatClass, keyClass, valueClass, conf) {
+
+  override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
+    hBaseContext.applyCreds()
+    super.compute(theSplit, context)
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/00248656/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
index f288c34..d859957 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.spark.datasources
 import java.util.ArrayList
 
 import org.apache.hadoop.hbase.client._
-import org.apache.hadoop.hbase.spark.{ScanRange, SchemaQualifierDefinition, HBaseRelation, SparkSQLPushDownFilter}
+import org.apache.hadoop.hbase.spark._
 import org.apache.hadoop.hbase.spark.hbase._
 import org.apache.hadoop.hbase.spark.datasources.HBaseResources._
 import org.apache.spark.{SparkEnv, TaskContext, Logging, Partition}
@@ -28,10 +28,10 @@ import org.apache.spark.rdd.RDD
 
 import scala.collection.mutable
 
-
 class HBaseTableScanRDD(relation: HBaseRelation,
-     @transient val filter: Option[SparkSQLPushDownFilter] = None,
-     val columns: Seq[SchemaQualifierDefinition] = Seq.empty
+                       val hbaseContext: HBaseContext,
+                       @transient val filter: Option[SparkSQLPushDownFilter] = None,
+                       val columns: Seq[SchemaQualifierDefinition] = Seq.empty
      )extends RDD[Result](relation.sqlContext.sparkContext, Nil) with Logging  {
   private def sparkConf = SparkEnv.get.conf
   @transient var ranges = Seq.empty[Range]
@@ -98,7 +98,8 @@ class HBaseTableScanRDD(relation: HBaseRelation,
       tbr: TableResource,
       g: Seq[Array[Byte]],
       filter: Option[SparkSQLPushDownFilter],
-      columns: Seq[SchemaQualifierDefinition]): Iterator[Result] = {
+      columns: Seq[SchemaQualifierDefinition],
+      hbaseContext: HBaseContext): Iterator[Result] = {
     g.grouped(relation.bulkGetSize).flatMap{ x =>
       val gets = new ArrayList[Get]()
       x.foreach{ y =>
@@ -111,6 +112,7 @@ class HBaseTableScanRDD(relation: HBaseRelation,
         filter.foreach(g.setFilter(_))
         gets.add(g)
       }
+      hbaseContext.applyCreds()
       val tmp = tbr.get(gets)
       rddResources.addResource(tmp)
       toResultIterator(tmp)
@@ -208,11 +210,12 @@ class HBaseTableScanRDD(relation: HBaseRelation,
       if (points.isEmpty) {
         Iterator.empty: Iterator[Result]
       } else {
-        buildGets(tableResource, points, filter, columns)
+        buildGets(tableResource, points, filter, columns, hbaseContext)
       }
     }
     val rIts = scans.par
       .map { scan =>
+      hbaseContext.applyCreds()
       val scanner = tableResource.getScanner(scan)
       rddResources.addResource(scanner)
       scanner


[09/37] hbase git commit: HBASE-15016 Services a Store needs from a Region

Posted by sy...@apache.org.
HBASE-15016 Services a Store needs from a Region

Signed-off-by: stack <st...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 876a6ab73ecff71b9b4010a532272474ea241daf
Parents: 28cd48b
Author: eshcar <es...@yahoo-inc.com>
Authored: Wed Feb 24 09:56:25 2016 +0200
Committer: stack <st...@apache.org>
Committed: Wed Feb 24 07:07:07 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/ClassSize.java |  4 +
 .../hbase/regionserver/DefaultMemStore.java     |  4 +
 .../hadoop/hbase/regionserver/HMobStore.java    |  3 +
 .../hadoop/hbase/regionserver/HRegion.java      | 93 +++++++++++++++-----
 .../hadoop/hbase/regionserver/HStore.java       |  4 +
 .../hadoop/hbase/regionserver/MemStore.java     |  7 ++
 .../hadoop/hbase/regionserver/Region.java       | 12 +--
 .../regionserver/RegionServicesForStores.java   | 53 +++++++++++
 .../apache/hadoop/hbase/regionserver/Store.java |  8 ++
 .../org/apache/hadoop/hbase/TestIOFencing.java  | 10 ++-
 10 files changed, 165 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
index 77acf9b..fdd0fae 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
@@ -110,6 +110,8 @@ public class ClassSize {
   /** Overhead for CellSkipListSet */
   public static final int CELL_SKIPLIST_SET;
 
+  public static final int STORE_SERVICES;
+
   /* Are we running on jdk7? */
   private static final boolean JDK7;
   static {
@@ -193,6 +195,8 @@ public class ClassSize {
     TIMERANGE_TRACKER = align(ClassSize.OBJECT + Bytes.SIZEOF_LONG * 2);
 
     CELL_SKIPLIST_SET = align(OBJECT + REFERENCE);
+
+    STORE_SERVICES = align(OBJECT + REFERENCE + ATOMIC_LONG);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index 82d40b6..92bb7b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -162,6 +162,10 @@ public class DefaultMemStore extends AbstractMemStore {
     return;
   }
 
+  @Override
+  public void finalizeFlush() {
+  }
+
   /**
    * Code to help figure if our approximation of object heap sizes is close
    * enough.  See hbase-900.  Fills memstores then waits so user can heap

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index d666db5..7b44338 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -511,6 +511,9 @@ public class HMobStore extends HStore {
     }
   }
 
+  @Override public void finalizeFlush() {
+  }
+
   public void updateCellsCountCompactedToMob(long count) {
     cellsCountCompactedToMob += count;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/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 0d5a71e..b70a4c3 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
@@ -17,6 +17,20 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Closeables;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import com.google.protobuf.TextFormat;
+
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -181,20 +195,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Closeables;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-import com.google.protobuf.TextFormat;
-
 @SuppressWarnings("deprecation")
 @InterfaceAudience.Private
 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
@@ -258,6 +258,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
 
   private final AtomicLong memstoreSize = new AtomicLong(0);
+  private final RegionServicesForStores regionServicesForStores = new RegionServicesForStores(this);
 
   // Debug possible data loss due to WAL off
   final Counter numMutationsWithoutWAL = new Counter();
@@ -999,6 +1000,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return false;
   }
 
+  public void blockUpdates() {
+    this.updatesLock.writeLock().lock();
+  }
+
+  public void unblockUpdates() {
+    this.updatesLock.writeLock().unlock();
+  }
+
   @Override
   public HDFSBlocksDistribution getHDFSBlocksDistribution() {
     HDFSBlocksDistribution hdfsBlocksDistribution =
@@ -1116,6 +1125,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return memstoreSize.get();
   }
 
+  public RegionServicesForStores getRegionServicesForStores() {
+    return regionServicesForStores;
+  }
+
   @Override
   public long getNumMutationsWithoutWAL() {
     return numMutationsWithoutWAL.get();
@@ -2035,7 +2048,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * Should the store be flushed because it is old enough.
    * <p>
    * Every FlushPolicy should call this to determine whether a store is old enough to flush (except
-   * that you always flush all stores). Otherwise the {@link #shouldFlush()} method will always
+   * that you always flush all stores). Otherwise the method will always
    * returns true which will make a lot of flush requests.
    */
   boolean shouldFlushStore(Store store) {
@@ -2477,6 +2490,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
 
     // If we get to here, the HStores have been written.
+    for(Store storeToFlush :storesToFlush) {
+      storeToFlush.finalizeFlush();
+    }
     if (wal != null) {
       wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
     }
@@ -2883,9 +2899,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
         long addedSize = doMiniBatchMutate(batchOp);
         long newSize = this.addAndGetGlobalMemstoreSize(addedSize);
-        if (isFlushSize(newSize)) {
-          requestFlush();
-        }
+        requestFlushIfNeeded(newSize);
       }
     } finally {
       closeRegionOperation(op);
@@ -3762,6 +3776,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  private void requestFlushIfNeeded(long memstoreTotalSize) throws RegionTooBusyException {
+    if(memstoreTotalSize > this.getMemstoreFlushSize()) {
+      requestFlush();
+    }
+  }
+
   private void requestFlush() {
     if (this.rsServices == null) {
       return;
@@ -5170,7 +5190,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       long c = count.decrementAndGet();
       if (c <= 0) {
         synchronized (lock) {
-          if (count.get() <= 0 ){
+          if (count.get() <= 0){
             usable.set(false);
             RowLockContext removed = lockedRows.remove(row);
             assert removed == this: "we should never remove a different context";
@@ -5978,7 +5998,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     protected boolean isStopRow(Cell currentRowCell) {
       return currentRowCell == null
-          || (stopRow != null && comparator.compareRows(currentRowCell, stopRow, 0, stopRow.length) >= isScan);
+          || (stopRow != null && comparator.compareRows(currentRowCell, stopRow, 0, stopRow
+          .length) >= isScan);
     }
 
     @Override
@@ -6860,8 +6881,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       processor.postProcess(this, walEdit, success);
     } finally {
       closeRegionOperation();
-      if (!mutations.isEmpty() && isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize))) {
-        requestFlush();
+      if (!mutations.isEmpty()) {
+        long newSize = this.addAndGetGlobalMemstoreSize(addedSize);
+        requestFlushIfNeeded(newSize);
       }
     }
   }
@@ -7290,7 +7312,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   /**
    * Do a specific Get on passed <code>columnFamily</code> and column qualifiers.
    * @param mutation Mutation we are doing this Get for.
-   * @param columnFamily Which column family on row (TODO: Go all Gets in one go)
+   * @param store Which column family on row (TODO: Go all Gets in one go)
    * @param coordinates Cells from <code>mutation</code> used as coordinates applied to Get.
    * @return Return list of Cells found.
    */
@@ -7340,7 +7362,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      45 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
+      46 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
       (14 * Bytes.SIZEOF_LONG) +
       5 * Bytes.SIZEOF_BOOLEAN);
 
@@ -7365,6 +7387,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       MultiVersionConcurrencyControl.FIXED_SIZE // mvcc
       + ClassSize.TREEMAP // maxSeqIdInStores
       + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress
+      + ClassSize.STORE_SERVICES // store services
       ;
 
   @Override
@@ -7847,4 +7870,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public long getMemstoreFlushSize() {
     return this.memstoreFlushSize;
   }
+
+  //// method for debugging tests
+  void throwException(String title, String regionName) {
+    StringBuffer buf = new StringBuffer();
+    buf.append(title + ", ");
+    buf.append(getRegionInfo().toString());
+    buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " ");
+    buf.append(getRegionInfo().isMetaTable() ? " meta table " : " ");
+    buf.append("stores: ");
+    for (Store s : getStores()) {
+      buf.append(s.getFamily().getNameAsString());
+      buf.append(" size: ");
+      buf.append(s.getMemStoreSize());
+      buf.append(" ");
+    }
+    buf.append("end-of-stores");
+    buf.append(", memstore size ");
+    buf.append(getMemstoreSize());
+    if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) {
+      throw new RuntimeException(buf.toString());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 5cc3fc9..22f99e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -2448,6 +2448,10 @@ public class HStore implements Store {
     }
   }
 
+  @Override public void finalizeFlush() {
+    memstore.finalizeFlush();
+  }
+
   private void clearCompactedfiles(final List<StoreFile> filesToRemove) throws IOException {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Clearing the compacted file " + filesToRemove + " from this store");

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
index a10ccd9..6bb7081 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
@@ -144,4 +144,11 @@ public interface MemStore extends HeapSize {
    * @return Total memory occupied by this MemStore.
    */
   long size();
+
+  /**
+   * This method is called when it is clear that the flush to disk is completed.
+   * The store may do any post-flush actions at this point.
+   * One example is to update the wal with sequence number that is known only at the store level.
+   */
+  void finalizeFlush();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 976bddb..9b1f82a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
@@ -49,11 +53,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServic
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-
 /**
  * Regions store data for a certain region of a table.  It stores all columns
  * for each row. A given table consists of one or more Regions.
@@ -200,6 +199,9 @@ public interface Region extends ConfigurationObserver {
   /** @return memstore size for this region, in bytes */
   long getMemstoreSize();
 
+  /** @return store services for this region, to access services required by store level needs */
+  RegionServicesForStores getRegionServicesForStores();
+
   /** @return the number of mutations processed bypassing the WAL */
   long getNumMutationsWithoutWAL();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
new file mode 100644
index 0000000..d3c35b3
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -0,0 +1,53 @@
+/*
+ *
+ * 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;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Services a Store needs from a Region.
+ * RegionServicesForStores class is the interface through which memstore access services at the
+ * region level.
+ * For example, when using alternative memory formats or due to compaction the memstore needs to
+ * take occasional lock and update size counters at the region level.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RegionServicesForStores {
+
+  private final HRegion region;
+
+  public RegionServicesForStores(HRegion region) {
+    this.region = region;
+  }
+
+  public void blockUpdates() {
+    this.region.blockUpdates();
+  }
+
+  public void unblockUpdates() {
+    this.region.unblockUpdates();
+  }
+
+  public long addAndGetGlobalMemstoreSize(long size) {
+    return this.region.addAndGetGlobalMemstoreSize(size);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index 09e0254..c167535 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -515,4 +515,12 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
    * Closes and archives the compacted files under this store
    */
   void closeAndArchiveCompactedFiles() throws IOException;
+
+  /**
+   * This method is called when it is clear that the flush to disk is completed.
+   * The store may do any post-flush actions at this point.
+   * One example is to update the wal with sequence number that is known only at the store level.
+   */
+  void finalizeFlush();
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/876a6ab7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index 35a7403..3aae5d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -17,14 +17,12 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -54,7 +52,8 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Test for the case where a regionserver going down has enough cycles to do damage to regions
@@ -206,6 +205,9 @@ public class TestIOFencing {
       }
       super.completeCompaction(compactedFiles);
     }
+
+    @Override public void finalizeFlush() {
+    }
   }
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();


[34/37] hbase git commit: HBASE-15348 Disable metrics tests until fixed.

Posted by sy...@apache.org.
HBASE-15348 Disable metrics tests until fixed.


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

Branch: refs/heads/hbase-12439
Commit: e88d94318321d40993953180368d33d24602a2ae
Parents: 8f2bd06
Author: Elliott Clark <ec...@apache.org>
Authored: Fri Feb 26 09:04:18 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Fri Feb 26 09:04:51 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java    | 2 ++
 .../hadoop/hbase/regionserver/TestRegionServerMetrics.java       | 4 +++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e88d9431/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
index 1f149bf..d4f7cdd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
@@ -50,12 +50,14 @@ import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.FixMethodOrder;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runners.MethodSorters;
 
 @Category({ MiscTests.class, MediumTests.class })
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@Ignore
 public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
   private static final Log LOG = LogFactory.getLog(TestStochasticBalancerJmxMetrics.class);
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e88d9431/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 7575e7b..1ec0bf7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -29,6 +29,7 @@ import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -513,6 +514,7 @@ public class TestRegionServerMetrics {
   }
   
   @Test
+  @Ignore
   public void testRangeCountMetrics() throws Exception {
     String tableNameString = "testRangeCountMetrics";
     final long[] timeranges =
@@ -558,7 +560,7 @@ public class TestRegionServerMetrics {
       dynamicMetricName =
           timeRangeMetricName + "_" + timeRangeType + "_" + prior + "-" + timeranges[i];
       if (metricsHelper.checkCounterExists(dynamicMetricName, serverSource)) {
-        long count = metricsHelper.getCounter(dynamicMetricName, serverSource);
+        long count = metricsHelper.getGaugeLong(dynamicMetricName, serverSource);
         if (count > 0) {
           timeRangeCountUpdated = true;
           break;


[02/37] hbase git commit: HBASE-15301 Remove the never-thrown NamingException from TableInputFormatBase#reverseDNS method signature (Yu Li)

Posted by sy...@apache.org.
HBASE-15301 Remove the never-thrown NamingException from TableInputFormatBase#reverseDNS method signature (Yu Li)


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

Branch: refs/heads/hbase-12439
Commit: 5e501123ca7854b18c63674029cdf3e7aeac5cf9
Parents: 2d66cd8
Author: stack <st...@apache.org>
Authored: Mon Feb 22 21:58:20 2016 -0800
Committer: stack <st...@apache.org>
Committed: Mon Feb 22 21:58:20 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/TableInputFormatBase.java     | 11 ++---------
 .../hadoop/hbase/mapreduce/TestTableInputFormatBase.java |  4 +---
 2 files changed, 3 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5e501123/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index b2f115c..9b5bbfb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -27,8 +27,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 
-import javax.naming.NamingException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -285,12 +283,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
         }
         InetAddress regionAddress = isa.getAddress();
         String regionLocation;
-        try {
-          regionLocation = reverseDNS(regionAddress);
-        } catch (NamingException e) {
-          LOG.warn("Cannot resolve the host name for " + regionAddress + " because of " + e);
-          regionLocation = location.getHostname();
-        }
+        regionLocation = reverseDNS(regionAddress);
   
         byte[] startRow = scan.getStartRow();
         byte[] stopRow = scan.getStopRow();
@@ -344,7 +337,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
     }
   }
 
-  String reverseDNS(InetAddress ipAddress) throws NamingException, UnknownHostException {
+  String reverseDNS(InetAddress ipAddress) throws UnknownHostException {
     String hostName = this.reverseDNSCacheMap.get(ipAddress);
     if (hostName == null) {
       String ipAddressString = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/5e501123/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
index c757a2d..699e773 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
@@ -24,8 +24,6 @@ import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 
-import javax.naming.NamingException;
-
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -34,7 +32,7 @@ import org.junit.experimental.categories.Category;
 public class TestTableInputFormatBase {
   @Test
   public void testTableInputFormatBaseReverseDNSForIPv6()
-      throws UnknownHostException, NamingException {
+      throws UnknownHostException {
     String address = "ipv6.google.com";
     String localhost = null;
     InetAddress addr = null;


[05/37] hbase git commit: HBASE-15291 FileSystem not closed in secure bulkLoad (Yong Zhang)

Posted by sy...@apache.org.
HBASE-15291 FileSystem not closed in secure bulkLoad (Yong Zhang)


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

Branch: refs/heads/hbase-12439
Commit: 58283fa1b1b10beec62cefa40babff6a1424b06c
Parents: 3ba1a7f
Author: tedyu <yu...@gmail.com>
Authored: Tue Feb 23 06:11:39 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Feb 23 06:11:39 2016 -0800

----------------------------------------------------------------------
 .../hbase/security/access/SecureBulkLoadEndpoint.java     | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/58283fa1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
index 06a2298..710ab44 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
@@ -292,6 +292,16 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
                 new SecureBulkLoadListener(fs, bulkToken, conf));
           } catch (Exception e) {
             LOG.error("Failed to complete bulk load", e);
+          } finally {
+            if (fs != null) {
+              try {
+                if(!UserGroupInformation.getCurrentUser().equals(ugi)) {
+                  FileSystem.closeAllForUGI(ugi);
+                }
+              } catch (IOException e) {
+                LOG.error("Failed to close FileSystem for " + ugi.getUserName(), e);
+              }
+            }
           }
           return false;
         }


[20/37] hbase git commit: HBASE-15222 Addendum - Use less contended classes for metrics

Posted by sy...@apache.org.
HBASE-15222 Addendum - Use less contended classes for metrics


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

Branch: refs/heads/hbase-12439
Commit: 77133fd225df9f65be87ce97b38676d2bab48a71
Parents: 43f99de
Author: Elliott Clark <ec...@apache.org>
Authored: Thu Feb 25 09:08:11 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Thu Feb 25 09:08:11 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/FastLongHistogram.java   | 10 +++-------
 .../org/apache/hadoop/metrics2/lib/MutableHistogram.java  |  4 +++-
 .../apache/hadoop/metrics2/lib/MutableRangeHistogram.java |  6 ++++--
 3 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/77133fd2/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
index 78b2bf0..9b403d9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
@@ -310,12 +310,8 @@ public class FastLongHistogram {
    * Resets the histogram for new counting.
    */
   public FastLongHistogram reset() {
-    if (this.bins.hasData.get()) {
-      Bins oldBins = this.bins;
-      this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
-      return new FastLongHistogram(oldBins);
-    }
-
-    return null;
+    Bins oldBins = this.bins;
+    this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
+    return new FastLongHistogram(oldBins);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/77133fd2/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
index 717e0ee..5b4a294 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
@@ -63,7 +63,9 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
   public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
     // Get a reference to the old histogram.
     FastLongHistogram histo = histogram.reset();
-    updateSnapshotMetrics(metricsRecordBuilder, histo);
+    if (histo != null) {
+      updateSnapshotMetrics(metricsRecordBuilder, histo);
+    }
   }
 
   protected void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder,

http://git-wip-us.apache.org/repos/asf/hbase/blob/77133fd2/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
index ac8aee0..13187af 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
@@ -56,8 +56,10 @@ public abstract class MutableRangeHistogram extends MutableHistogram implements
   public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
     // Get a reference to the old histogram.
     FastLongHistogram histo = histogram.reset();
-    updateSnapshotMetrics(metricsRecordBuilder, histo);
-    updateSnapshotRangeMetrics(metricsRecordBuilder, histo);
+    if (histo != null) {
+      updateSnapshotMetrics(metricsRecordBuilder, histo);
+      updateSnapshotRangeMetrics(metricsRecordBuilder, histo);
+    }
   }
 
   public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder,


[33/37] hbase git commit: HBASE-15205 Do not find the replication scope for every WAL#append() (Ram)

Posted by sy...@apache.org.
HBASE-15205 Do not find the replication scope for every WAL#append() (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 8f2bd06019869a1738bcfd66066737cdb7802ca8
Parents: 538815d
Author: ramkrishna <ra...@gmail.com>
Authored: Fri Feb 26 22:30:55 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Fri Feb 26 22:30:55 2016 +0530

----------------------------------------------------------------------
 .../hbase/protobuf/ReplicationProtbufUtil.java  |   2 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  89 +++++++++-----
 .../hadoop/hbase/regionserver/HStore.java       |   2 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  13 +-
 .../hbase/regionserver/wal/FSWALEntry.java      |  10 +-
 .../hadoop/hbase/regionserver/wal/HLogKey.java  |  48 ++++++--
 .../regionserver/wal/WALActionsListener.java    |   8 +-
 .../hadoop/hbase/regionserver/wal/WALUtil.java  |  57 +++++----
 .../hbase/replication/ScopeWALEntryFilter.java  |   2 +-
 .../replication/regionserver/Replication.java   |  70 +++--------
 .../hadoop/hbase/wal/DisabledWALProvider.java   |   4 +-
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |   9 +-
 .../org/apache/hadoop/hbase/wal/WALKey.java     | 121 +++++++++++++++----
 .../apache/hadoop/hbase/wal/WALSplitter.java    |   2 +-
 .../org/apache/hadoop/hbase/TestIOFencing.java  |   3 +-
 .../hbase/coprocessor/TestWALObserver.java      |  48 +++++---
 .../hbase/mapreduce/TestHLogRecordReader.java   |   7 +-
 .../hbase/mapreduce/TestImportExport.java       |  16 +--
 .../hbase/mapreduce/TestWALRecordReader.java    |  20 +--
 .../master/TestDistributedLogSplitting.java     |   9 +-
 .../hadoop/hbase/regionserver/TestBulkLoad.java |  17 +--
 .../hadoop/hbase/regionserver/TestHRegion.java  |  16 +--
 .../regionserver/TestHRegionReplayEvents.java   |   6 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   3 +-
 .../hbase/regionserver/TestWALLockup.java       |  10 +-
 .../hbase/regionserver/wal/TestFSHLog.java      |  57 ++++++---
 .../regionserver/wal/TestLogRollAbort.java      |  12 +-
 .../wal/TestLogRollingNoCluster.java            |  11 +-
 .../wal/TestWALActionsListener.java             |  12 +-
 .../hbase/regionserver/wal/TestWALReplay.java   |  47 ++++---
 .../hbase/replication/TestReplicationBase.java  |   9 ++
 .../replication/TestReplicationSmallTests.java  |  13 +-
 .../TestReplicationWALEntryFilters.java         |  62 +++++-----
 .../TestReplicationSourceManager.java           |  57 +++++----
 .../TestReplicationWALReaderManager.java        |  13 +-
 .../apache/hadoop/hbase/wal/FaultyFSLog.java    |   7 +-
 .../hbase/wal/TestDefaultWALProvider.java       |  64 +++++++---
 .../wal/TestDefaultWALProviderWithHLogKey.java  |   7 +-
 .../apache/hadoop/hbase/wal/TestSecureWAL.java  |  11 +-
 .../apache/hadoop/hbase/wal/TestWALFactory.java |  74 ++++++++----
 .../hbase/wal/TestWALReaderOnSecureWAL.java     |  11 +-
 .../hbase/wal/WALPerformanceEvaluation.java     |  15 ++-
 42 files changed, 685 insertions(+), 389 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index 91185af..8cb2237 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -134,7 +134,7 @@ public class ReplicationProtbufUtil {
         keyBuilder.setOrigSequenceNumber(key.getOrigLogSeqNum());
       }
       WALEdit edit = entry.getEdit();
-      NavigableMap<byte[], Integer> scopes = key.getScopes();
+      NavigableMap<byte[], Integer> scopes = key.getReplicationScopes();
       if (scopes != null && !scopes.isEmpty()) {
         for (Map.Entry<byte[], Integer> scope: scopes.entrySet()) {
           scopeBuilder.setFamily(ByteStringer.wrap(scope.getKey()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 b70a4c3..406850e 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
@@ -17,19 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Closeables;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-import com.google.protobuf.TextFormat;
+import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
 
 import java.io.EOFException;
 import java.io.FileNotFoundException;
@@ -195,6 +183,20 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Closeables;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import com.google.protobuf.TextFormat;
+
 @SuppressWarnings("deprecation")
 @InterfaceAudience.Private
 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
@@ -583,6 +585,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private final MetricsRegionWrapperImpl metricsRegionWrapper;
   private final Durability durability;
   private final boolean regionStatsEnabled;
+  // Stores the replication scope of the various column families of the table
+  // that has non-default scope
+  private final NavigableMap<byte[], Integer> replicationScope = new TreeMap<byte[], Integer>(
+      Bytes.BYTES_COMPARATOR);
 
   /**
    * HRegion constructor. This constructor should only be used for testing and
@@ -661,6 +667,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
     this.htableDescriptor = htd;
+    Set<byte[]> families = this.htableDescriptor.getFamiliesKeys();
+    for (byte[] family : families) {
+      if (!replicationScope.containsKey(family)) {
+        int scope = htd.getFamily(family).getScope();
+        // Only store those families that has NON-DEFAULT scope
+        if (scope != REPLICATION_SCOPE_LOCAL) {
+          // Do a copy before storing it here.
+          replicationScope.put(Bytes.copy(family), scope);
+        }
+      }
+    }
     this.rsServices = rsServices;
     this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
     setHTableSpecificConf();
@@ -971,7 +988,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
       RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId,
       getRegionServerServices().getServerName(), storeFiles);
-    WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionOpenDesc, mvcc);
+    WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionOpenDesc,
+        mvcc);
   }
 
   private void writeRegionCloseMarker(WAL wal) throws IOException {
@@ -979,7 +997,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor(
       RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(),
       getRegionServerServices().getServerName(), storeFiles);
-    WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionEventDesc, mvcc);
+    WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionEventDesc,
+        mvcc);
 
     // Store SeqId in HDFS when a region closes
     // checking region folder exists is due to many tests which delete the table folder while a
@@ -2285,7 +2304,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
             getRegionInfo(), flushOpSeqId, committedFiles);
         // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH
-        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, false, mvcc);
+        WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false,
+            mvcc);
       }
 
       // Prepare flush (take a snapshot)
@@ -2334,7 +2354,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     try {
       FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
           getRegionInfo(), flushOpSeqId, committedFiles);
-      WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, false,
+      WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false,
           mvcc);
     } catch (Throwable t) {
       LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL:" +
@@ -2379,7 +2399,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH,
         getRegionInfo(), -1, new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR));
       try {
-        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, true, mvcc);
+        WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true,
+            mvcc);
         return true;
       } catch (IOException e) {
         LOG.warn(getRegionInfo().getEncodedName() + " : "
@@ -2449,7 +2470,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
         FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
           getRegionInfo(), flushOpSeqId, committedFiles);
-        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, true, mvcc);
+        WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true,
+            mvcc);
       }
     } catch (Throwable t) {
       // An exception here means that the snapshot was not persisted.
@@ -2462,7 +2484,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         try {
           FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
             getRegionInfo(), flushOpSeqId, committedFiles);
-          WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, false, mvcc);
+          WALUtil.writeFlushMarker(wal, this.replicationScope, getRegionInfo(), desc, false, mvcc);
         } catch (Throwable ex) {
           LOG.warn(getRegionInfo().getEncodedName() + " : "
               + "failed writing ABORT_FLUSH marker to WAL", ex);
@@ -3139,13 +3161,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (!replay) {
           // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
           walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
-            this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
-            mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
+              this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
+              mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc,
+              this.getReplicationScope());
         }
         // TODO: Use the doAppend methods below... complicated by the replay stuff above.
         try {
-          long txid =
-            this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
+          long txid = this.wal.append(this.getRegionInfo(), walKey,
+              walEdit, true);
           if (txid != 0) sync(txid, durability);
           writeEntry = walKey.getWriteEntry();
         } catch (IOException ioe) {
@@ -3271,8 +3294,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     if (!replay) throw new IOException("Multiple nonces per batch and not in replay");
     WALKey walKey = new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
         this.htableDescriptor.getTableName(), now, mutation.getClusterIds(),
-        currentNonceGroup, currentNonce, mvcc);
-    this.wal.append(this.htableDescriptor,  this.getRegionInfo(), walKey, walEdit, true);
+        currentNonceGroup, currentNonce, mvcc, this.getReplicationScope());
+    this.wal.append(this.getRegionInfo(), walKey, walEdit, true);
     // Complete the mvcc transaction started down in append else it will block others
     this.mvcc.complete(walKey.getWriteEntry());
   }
@@ -5389,7 +5412,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           WALProtos.BulkLoadDescriptor loadDescriptor = ProtobufUtil.toBulkLoadDescriptor(
               this.getRegionInfo().getTable(),
               ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles, seqId);
-          WALUtil.writeBulkLoadMarkerAndSync(this.wal, getTableDesc(), getRegionInfo(),
+          WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(),
               loadDescriptor, mvcc);
         } catch (IOException ioe) {
           if (this.rsServices != null) {
@@ -6319,6 +6342,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return r.openHRegion(reporter);
   }
 
+  @VisibleForTesting
+  public NavigableMap<byte[], Integer> getReplicationScope() {
+    return this.replicationScope;
+  }
 
   /**
    * Useful when reopening a closed region (normally for unit tests)
@@ -7069,10 +7096,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // here instead of WALKey directly to support legacy coprocessors.
     WALKey walKey = new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
       this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now, clusterIds,
-      nonceGroup, nonce, mvcc);
+      nonceGroup, nonce, mvcc, this.getReplicationScope());
     try {
       long txid =
-        this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
+        this.wal.append(this.getRegionInfo(), walKey, walEdit, true);
       // Call sync on our edit.
       if (txid != 0) sync(txid, durability);
       writeEntry = walKey.getWriteEntry();
@@ -7362,7 +7389,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      46 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
+      47 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
       (14 * Bytes.SIZEOF_LONG) +
       5 * Bytes.SIZEOF_BOOLEAN);
 
@@ -7385,7 +7412,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
       (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
       MultiVersionConcurrencyControl.FIXED_SIZE // mvcc
-      + ClassSize.TREEMAP // maxSeqIdInStores
+      + 2 * ClassSize.TREEMAP // maxSeqIdInStores, replicationScopes
       + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress
       + ClassSize.STORE_SERVICES // store services
       ;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 22f99e9..7c71baf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1307,7 +1307,7 @@ public class HStore implements Store {
     // Fix reaching into Region to get the maxWaitForSeqId.
     // Does this method belong in Region altogether given it is making so many references up there?
     // Could be Region#writeCompactionMarker(compactionDescriptor);
-    WALUtil.writeCompactionMarker(this.region.getWAL(), this.region.getTableDesc(),
+    WALUtil.writeCompactionMarker(this.region.getWAL(), this.region.getReplicationScope(),
         this.region.getRegionInfo(), compactionDescriptor, this.region.getMVCC());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 09da8fc..f3f869c 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
@@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -1083,8 +1082,8 @@ public class FSHLog implements WAL {
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
       justification="Will never be null")
   @Override
-  public long append(final HTableDescriptor htd, final HRegionInfo hri, final WALKey key,
-      final WALEdit edits, final boolean inMemstore) throws IOException {
+  public long append(final HRegionInfo hri,
+      final WALKey key, final WALEdit edits, final boolean inMemstore) throws IOException {
     if (this.closed) throw new IOException("Cannot append; log is closed");
     // Make a trace scope for the append.  It is closed on other side of the ring buffer by the
     // single consuming thread.  Don't have to worry about it.
@@ -1100,7 +1099,7 @@ public class FSHLog implements WAL {
       // Construction of FSWALEntry sets a latch.  The latch is thrown just after we stamp the
       // edit with its edit/sequence id.
       // TODO: reuse FSWALEntry as we do SyncFuture rather create per append.
-      entry = new FSWALEntry(sequence, key, edits, htd, hri, inMemstore);
+      entry = new FSWALEntry(sequence, key, edits, hri, inMemstore);
       truck.loadPayload(entry, scope.detach());
     } finally {
       this.disruptor.getRingBuffer().publish(sequence);
@@ -1878,14 +1877,12 @@ public class FSHLog implements WAL {
             entry.getEdit())) {
           if (entry.getEdit().isReplay()) {
             // Set replication scope null so that this won't be replicated
-            entry.getKey().setScopes(null);
+            entry.getKey().serializeReplicationScope(false);
           }
         }
         if (!listeners.isEmpty()) {
           for (WALActionsListener i: listeners) {
-            // TODO: Why does listener take a table description and CPs take a regioninfo?  Fix.
-            i.visitLogEntryBeforeWrite(entry.getHTableDescriptor(), entry.getKey(),
-              entry.getEdit());
+            i.visitLogEntryBeforeWrite(entry.getKey(), entry.getEdit());
           }
         }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 86a3c3d..06318f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -26,7 +26,6 @@ import java.util.Set;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -51,15 +50,13 @@ class FSWALEntry extends Entry {
   // they are only in memory and held here while passing over the ring buffer.
   private final transient long sequence;
   private final transient boolean inMemstore;
-  private final transient HTableDescriptor htd;
   private final transient HRegionInfo hri;
   private final Set<byte[]> familyNames;
 
   FSWALEntry(final long sequence, final WALKey key, final WALEdit edit,
-      final HTableDescriptor htd, final HRegionInfo hri, final boolean inMemstore) {
+      final HRegionInfo hri, final boolean inMemstore) {
     super(key, edit);
     this.inMemstore = inMemstore;
-    this.htd = htd;
     this.hri = hri;
     this.sequence = sequence;
     if (inMemstore) {
@@ -71,6 +68,7 @@ class FSWALEntry extends Entry {
         Set<byte[]> familySet = Sets.newTreeSet(Bytes.BYTES_COMPARATOR);
         for (Cell cell : cells) {
           if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+            // TODO: Avoid this clone?
             familySet.add(CellUtil.cloneFamily(cell));
           }
         }
@@ -89,10 +87,6 @@ class FSWALEntry extends Entry {
     return this.inMemstore;
   }
 
-  HTableDescriptor getHTableDescriptor() {
-    return this.htd;
-  }
-
   HRegionInfo getHRegionInfo() {
     return this.hri;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
index 7c40323..d7bf4a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
@@ -24,6 +24,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
+import java.util.NavigableMap;
 import java.util.UUID;
 
 import org.apache.commons.logging.Log;
@@ -67,7 +68,7 @@ public class HLogKey extends WALKey implements Writable {
   }
 
   public HLogKey(final byte[] encodedRegionName, final TableName tablename) {
-    super(encodedRegionName, tablename);
+    super(encodedRegionName, tablename, null);
   }
 
   @VisibleForTesting
@@ -75,11 +76,15 @@ public class HLogKey extends WALKey implements Writable {
     super(encodedRegionName, tablename, now);
   }
 
-  public HLogKey(final byte[] encodedRegionName,
-                 final TableName tablename,
-                 final long now,
-                 final MultiVersionConcurrencyControl mvcc) {
-    super(encodedRegionName, tablename, now, mvcc);
+  @VisibleForTesting
+  public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now,
+      final NavigableMap<byte[], Integer> replicationScope) {
+    super(encodedRegionName, tablename, now, replicationScope);
+  }
+
+  public HLogKey(final byte[] encodedRegionName, final TableName tablename, final long now,
+      final MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> scopes) {
+    super(encodedRegionName, tablename, now, mvcc, scopes);
   }
 
   /**
@@ -111,6 +116,35 @@ public class HLogKey extends WALKey implements Writable {
    * Create the log key for writing to somewhere.
    * We maintain the tablename mainly for debugging purposes.
    * A regionName is always a sub-table object.
+   * <p>Used by log splitting and snapshots.
+   *
+   * @param encodedRegionName Encoded name of the region as returned by
+   * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
+   * @param tablename   - name of table
+   * @param logSeqNum   - log sequence number
+   * @param now Time at which this edit was written.
+   * @param clusterIds the clusters that have consumed the change(used in Replication)
+   * @param nonceGroup the noncegroup
+   * @param nonce      the nonce
+   * @param replicationScope the replicationScope of the non-default column families' of the region
+   */
+  public HLogKey(
+      final byte[] encodedRegionName,
+      final TableName tablename,
+      long logSeqNum,
+      final long now,
+      List<UUID> clusterIds,
+      long nonceGroup,
+      long nonce,
+      MultiVersionConcurrencyControl mvcc, NavigableMap<byte[], Integer> replicationScope) {
+    super(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc,
+        replicationScope);
+  }
+
+  /**
+   * Create the log key for writing to somewhere.
+   * We maintain the tablename mainly for debugging purposes.
+   * A regionName is always a sub-table object.
    *
    * @param encodedRegionName Encoded name of the region as returned by
    * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
@@ -192,7 +226,7 @@ public class HLogKey extends WALKey implements Writable {
     // encodes the length of encodedRegionName.
     // If < 0 we just read the version and the next vint is the length.
     // @see Bytes#readByteArray(DataInput)
-    setScopes(null); // writable HLogKey does not contain scopes
+    serializeReplicationScope(false); // writable HLogKey does not contain scopes
     int len = WritableUtils.readVInt(in);
     byte[] tablenameBytes = null;
     if (len < 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
index db98083..a6452e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 
 import org.apache.hadoop.hbase.wal.WALKey;
 
@@ -85,7 +84,6 @@ public interface WALActionsListener {
   );
 
   /**
-   * @param htd
    * @param logKey
    * @param logEdit TODO: Retire this in favor of
    *          {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)} It only exists to get
@@ -93,8 +91,7 @@ public interface WALActionsListener {
    *          <code>htd</code>.
    * @throws IOException If failed to parse the WALEdit
    */
-  void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
-      throws IOException;
+  void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException;
 
   /**
    * For notification post append to the writer.  Used by metrics system at least.
@@ -135,8 +132,7 @@ public interface WALActionsListener {
     public void visitLogEntryBeforeWrite(HRegionInfo info, WALKey logKey, WALEdit logEdit) {}
 
     @Override
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
-        throws IOException {
+    public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index f268422..197144d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -20,11 +20,11 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
+import java.util.NavigableMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -58,10 +58,11 @@ public class WALUtil {
    * <p>This write is for internal use only. Not for external client consumption.
    * @param mvcc Used by WAL to get sequence Id for the waledit.
    */
-  public static WALKey writeCompactionMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
-      final CompactionDescriptor c, MultiVersionConcurrencyControl mvcc)
+  public static WALKey writeCompactionMarker(WAL wal,
+      NavigableMap<byte[], Integer> replicationScope, HRegionInfo hri, final CompactionDescriptor c,
+      MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    WALKey walKey = writeMarker(wal, htd, hri, WALEdit.createCompaction(hri, c), mvcc);
+    WALKey walKey = writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
     }
@@ -73,11 +74,11 @@ public class WALUtil {
    *
    * <p>This write is for internal use only. Not for external client consumption.
    */
-  public static WALKey writeFlushMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
-      final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
-  throws IOException {
-    WALKey walKey =
-      doFullAppendTransaction(wal, htd, hri, WALEdit.createFlushWALEdit(hri, f), mvcc, sync);
+  public static WALKey writeFlushMarker(WAL wal, NavigableMap<byte[], Integer> replicationScope,
+      HRegionInfo hri, final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
+          throws IOException {
+    WALKey walKey = doFullAppendTransaction(wal, replicationScope, hri,
+        WALEdit.createFlushWALEdit(hri, f), mvcc, sync);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
     }
@@ -88,10 +89,12 @@ public class WALUtil {
    * Write a region open marker indicating that the region is opened.
    * This write is for internal use only. Not for external client consumption.
    */
-  public static WALKey writeRegionEventMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
+  public static WALKey writeRegionEventMarker(WAL wal,
+      NavigableMap<byte[], Integer> replicationScope, HRegionInfo hri,
       final RegionEventDescriptor r, final MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    WALKey walKey = writeMarker(wal, htd, hri, WALEdit.createRegionEventWALEdit(hri, r), mvcc);
+    WALKey walKey = writeMarker(wal, replicationScope, hri,
+        WALEdit.createRegionEventWALEdit(hri, r), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
     }
@@ -102,28 +105,30 @@ public class WALUtil {
    * Write a log marker that a bulk load has succeeded and is about to be committed.
    * This write is for internal use only. Not for external client consumption.
    * @param wal The log to write into.
-   * @param htd A description of the table that we are bulk loading into.
+   * @param replicationScope The replication scope of the families in the HRegion
    * @param hri A description of the region in the table that we are bulk loading into.
    * @param desc A protocol buffers based description of the client's bulk loading request
    * @return walKey with sequenceid filled out for this bulk load marker
    * @throws IOException We will throw an IOException if we can not append to the HLog.
    */
-  public static WALKey writeBulkLoadMarkerAndSync(final WAL wal, final HTableDescriptor htd,
-      final HRegionInfo hri, final WALProtos.BulkLoadDescriptor desc,
-      final MultiVersionConcurrencyControl mvcc)
-  throws IOException {
-    WALKey walKey = writeMarker(wal, htd, hri, WALEdit.createBulkLoadEvent(hri, desc), mvcc);
+  public static WALKey writeBulkLoadMarkerAndSync(final WAL wal,
+      final NavigableMap<byte[], Integer> replicationScope, final HRegionInfo hri,
+      final WALProtos.BulkLoadDescriptor desc, final MultiVersionConcurrencyControl mvcc)
+          throws IOException {
+    WALKey walKey = writeMarker(wal, replicationScope, hri, WALEdit.createBulkLoadEvent(hri, desc),
+        mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(desc));
     }
     return walKey;
   }
 
-  private static WALKey writeMarker(final WAL wal, final HTableDescriptor htd,
-      final HRegionInfo hri, final WALEdit edit, final MultiVersionConcurrencyControl mvcc)
+  private static WALKey writeMarker(final WAL wal,
+      final NavigableMap<byte[], Integer> replicationScope, final HRegionInfo hri,
+      final WALEdit edit, final MultiVersionConcurrencyControl mvcc)
   throws IOException {
     // If sync == true in below, then timeout is not used; safe to pass UNSPECIFIED_TIMEOUT
-    return doFullAppendTransaction(wal, htd, hri, edit, mvcc, true);
+    return doFullAppendTransaction(wal, replicationScope, hri, edit, mvcc, true);
   }
 
   /**
@@ -134,16 +139,16 @@ public class WALUtil {
    * <p>This write is for internal use only. Not for external client consumption.
    * @return WALKey that was added to the WAL.
    */
-  public static WALKey doFullAppendTransaction(final WAL wal, final HTableDescriptor htd,
-      final HRegionInfo hri, final WALEdit edit, final MultiVersionConcurrencyControl mvcc,
-      final boolean sync)
+  public static WALKey doFullAppendTransaction(final WAL wal,
+      final NavigableMap<byte[], Integer> replicationScope, final HRegionInfo hri,
+      final WALEdit edit, final MultiVersionConcurrencyControl mvcc, final boolean sync)
   throws IOException {
     // TODO: Pass in current time to use?
-    WALKey walKey =
-      new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), System.currentTimeMillis(), mvcc);
+    WALKey walKey = new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(),
+        System.currentTimeMillis(), mvcc, replicationScope);
     long trx = MultiVersionConcurrencyControl.NONE;
     try {
-      trx = wal.append(htd, hri, walKey, edit, false);
+      trx = wal.append(hri, walKey, edit, false);
       if (sync) {
         wal.sync(trx);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
index f97ec15..28a83dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
@@ -44,7 +44,7 @@ public class ScopeWALEntryFilter implements WALEntryFilter {
 
   @Override
   public Entry filter(Entry entry) {
-    NavigableMap<byte[], Integer> scopes = entry.getKey().getScopes();
+    NavigableMap<byte[], Integer> scopes = entry.getKey().getReplicationScopes();
     if (scopes == null || scopes.isEmpty()) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index a5d2446..bb4a5a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -20,13 +20,10 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
-import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -43,7 +40,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
@@ -61,7 +57,6 @@ import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.zookeeper.KeeperException;
 
@@ -257,72 +252,47 @@ public class Replication extends WALActionsListener.Base implements
   }
 
   @Override
-  public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
-      throws IOException {
-    scopeWALEdits(htd, logKey, logEdit, this.conf, this.getReplicationManager());
+  public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
+    scopeWALEdits(logKey, logEdit, this.conf, this.getReplicationManager());
   }
 
   /**
    * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
    * compaction WAL edits and if the scope is local.
-   * @param htd Descriptor used to find the scope to use
    * @param logKey Key that may get scoped according to its edits
    * @param logEdit Edits used to lookup the scopes
    * @param replicationManager Manager used to add bulk load events hfile references
    * @throws IOException If failed to parse the WALEdit
    */
-  public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey, WALEdit logEdit,
-      Configuration conf, ReplicationSourceManager replicationManager) throws IOException {
-    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
-    byte[] family;
+  public static void scopeWALEdits(WALKey logKey,
+      WALEdit logEdit, Configuration conf, ReplicationSourceManager replicationManager)
+          throws IOException {
     boolean replicationForBulkLoadEnabled = isReplicationForBulkLoadDataEnabled(conf);
+    byte[] family;
+    boolean foundOtherEdits = false;
     for (Cell cell : logEdit.getCells()) {
       if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
         if (replicationForBulkLoadEnabled && CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
-          scopeBulkLoadEdits(htd, replicationManager, scopes, logKey.getTablename(), cell);
+          try {
+            BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
+            for (StoreDescriptor s : bld.getStoresList()) {
+              family = s.getFamilyName().toByteArray();
+              addHFileRefsToQueue(replicationManager, logKey.getTablename(), family, s);
+            }
+          } catch (IOException e) {
+            LOG.error("Failed to get bulk load events information from the wal file.", e);
+            throw e;
+          }
         } else {
           // Skip the flush/compaction/region events
           continue;
         }
       } else {
-        family = CellUtil.cloneFamily(cell);
-        // Unexpected, has a tendency to happen in unit tests
-        assert htd.getFamily(family) != null;
-
-        if (!scopes.containsKey(family)) {
-          int scope = htd.getFamily(family).getScope();
-          if (scope != REPLICATION_SCOPE_LOCAL) {
-            scopes.put(family, scope);
-          }
-        }
+        foundOtherEdits = true;
       }
     }
-    if (!scopes.isEmpty() && !logEdit.isReplay()) {
-      logKey.setScopes(scopes);
-    }
-  }
-
-  private static void scopeBulkLoadEdits(HTableDescriptor htd,
-      ReplicationSourceManager replicationManager, NavigableMap<byte[], Integer> scopes,
-      TableName tableName, Cell cell) throws IOException {
-    byte[] family;
-    try {
-      BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
-      for (StoreDescriptor s : bld.getStoresList()) {
-        family = s.getFamilyName().toByteArray();
-        if (!scopes.containsKey(family)) {
-          int scope = htd.getFamily(family).getScope();
-          if (scope != REPLICATION_SCOPE_LOCAL) {
-            scopes.put(family, scope);
-            addHFileRefsToQueue(replicationManager, tableName, family, s);
-          }
-        } else {
-          addHFileRefsToQueue(replicationManager, tableName, family, s);
-        }
-      }
-    } catch (IOException e) {
-      LOG.error("Failed to get bulk load events information from the wal file.", e);
-      throw e;
+    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
+      logKey.serializeReplicationScope(false);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 0c41e77..c3d4b2c 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.util.FSUtils;
 
 // imports for things that haven't moved from regionserver.wal yet.
@@ -154,8 +153,7 @@ class DisabledWALProvider implements WALProvider {
     }
 
     @Override
-    public long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits,
-                       boolean inMemstore) {
+    public long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore) {
       if (!this.listeners.isEmpty()) {
         final long start = System.nanoTime();
         long len = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 d2b336e..0b83528 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
@@ -25,7 +25,6 @@ import java.util.Set;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 // imports we use from yet-to-be-moved regionsever.wal
@@ -106,21 +105,17 @@ public interface WAL {
    * completes BUT on return this edit must have its region edit/sequence id assigned
    * else it messes up our unification of mvcc and sequenceid.  On return <code>key</code> will
    * have the region edit/sequence id filled in.
-   * @param info
+   * @param info the regioninfo associated with append
    * @param key Modified by this call; we add to it this edits region edit/sequence id.
    * @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
    * sequence id that is after all currently appended edits.
-   * @param htd used to give scope for replication TODO refactor out in favor of table name and
-   * info
    * @param inMemstore Always true except for case where we are writing a compaction completion
    * record into the WAL; in this case the entry is just so we can finish an unfinished compaction
    * -- it is not an edit for memstore.
    * @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
    * in it.
    */
-  long append(HTableDescriptor htd, HRegionInfo info, WALKey key, WALEdit edits,
-    boolean inMemstore)
-  throws IOException;
+  long append(HRegionInfo info, WALKey key, WALEdit edits, boolean inMemstore) throws IOException;
 
   /**
    * Sync what we have in the WAL.

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index 09096fe..86fdfbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -193,7 +193,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   @InterfaceAudience.Private
   protected List<UUID> clusterIds;
 
-  private NavigableMap<byte[], Integer> scopes;
+  private NavigableMap<byte[], Integer> replicationScope;
 
   private long nonceGroup = HConstants.NO_NONCE;
   private long nonce = HConstants.NO_NONCE;
@@ -210,7 +210,12 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
 
   public WALKey() {
     init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
-        new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE, null);
+        new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE, null, null);
+  }
+
+  public WALKey(final NavigableMap<byte[], Integer> replicationScope) {
+    init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
+        new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE, null, replicationScope);
   }
 
   @VisibleForTesting
@@ -220,15 +225,16 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     List<UUID> clusterIds = new ArrayList<UUID>();
     clusterIds.add(clusterId);
     init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
-        HConstants.NO_NONCE, HConstants.NO_NONCE, null);
+        HConstants.NO_NONCE, HConstants.NO_NONCE, null, null);
   }
 
   /**
    * @deprecated Remove. Useless.
    */
   @Deprecated // REMOVE
-  public WALKey(final byte[] encodedRegionName, final TableName tablename) {
-    this(encodedRegionName, tablename, System.currentTimeMillis());
+  public WALKey(final byte[] encodedRegionName, final TableName tablename,
+      final NavigableMap<byte[], Integer> replicationScope) {
+    this(encodedRegionName, tablename, System.currentTimeMillis(), replicationScope);
   }
 
   // TODO: Fix being able to pass in sequenceid.
@@ -240,7 +246,20 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
         EMPTY_UUIDS,
         HConstants.NO_NONCE,
         HConstants.NO_NONCE,
-        null);
+        null, null);
+  }
+
+  // TODO: Fix being able to pass in sequenceid.
+  public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now,
+      final NavigableMap<byte[], Integer> replicationScope) {
+    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
+        HConstants.NO_NONCE, null, replicationScope);
+  }
+
+  public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now,
+      MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> replicationScope) {
+    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
+        HConstants.NO_NONCE, mvcc, replicationScope);
   }
 
   public WALKey(final byte[] encodedRegionName,
@@ -254,7 +273,33 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
         EMPTY_UUIDS,
         HConstants.NO_NONCE,
         HConstants.NO_NONCE,
-        mvcc);
+        mvcc, null);
+  }
+
+  /**
+   * Create the log key for writing to somewhere.
+   * We maintain the tablename mainly for debugging purposes.
+   * A regionName is always a sub-table object.
+   * <p>Used by log splitting and snapshots.
+   *
+   * @param encodedRegionName Encoded name of the region as returned by
+   *                         <code>HRegionInfo#getEncodedNameAsBytes()</code>.
+   * @param tablename         - name of table
+   * @param logSeqNum         - log sequence number
+   * @param now               Time at which this edit was written.
+   * @param clusterIds        the clusters that have consumed the change(used in Replication)
+   * @param nonceGroup        the nonceGroup
+   * @param nonce             the nonce
+   * @param mvcc              the mvcc associate the WALKey
+   * @param replicationScope  the non-default replication scope
+   *                          associated with the region's column families
+   */
+  // TODO: Fix being able to pass in sequenceid.
+  public WALKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
+      final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
+      MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> replicationScope) {
+    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc,
+        replicationScope);
   }
 
   /**
@@ -279,7 +324,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
                 long nonceGroup,
                 long nonce,
                 MultiVersionConcurrencyControl mvcc) {
-    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc);
+    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc, null);
   }
 
   /**
@@ -289,7 +334,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
    *
    * @param encodedRegionName Encoded name of the region as returned by
    *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename
+   * @param tablename         the tablename
    * @param now               Time at which this edit was written.
    * @param clusterIds        the clusters that have consumed the change(used in Replication)
    * @param nonceGroup
@@ -299,7 +344,31 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   public WALKey(final byte[] encodedRegionName, final TableName tablename,
                 final long now, List<UUID> clusterIds, long nonceGroup,
                 final long nonce, final MultiVersionConcurrencyControl mvcc) {
-    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc);
+    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc,
+        null);
+  }
+
+  /**
+   * Create the log key for writing to somewhere.
+   * We maintain the tablename mainly for debugging purposes.
+   * A regionName is always a sub-table object.
+   *
+   * @param encodedRegionName Encoded name of the region as returned by
+   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
+   * @param tablename
+   * @param now               Time at which this edit was written.
+   * @param clusterIds        the clusters that have consumed the change(used in Replication)
+   * @param nonceGroup        the nonceGroup
+   * @param nonce             the nonce
+   * @param mvcc mvcc control used to generate sequence numbers and control read/write points
+   * @param replicationScope  the non-default replication scope of the column families
+   */
+  public WALKey(final byte[] encodedRegionName, final TableName tablename,
+                final long now, List<UUID> clusterIds, long nonceGroup,
+                final long nonce, final MultiVersionConcurrencyControl mvcc,
+                NavigableMap<byte[], Integer> replicationScope) {
+    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc,
+        replicationScope);
   }
 
   /**
@@ -328,7 +397,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
         EMPTY_UUIDS,
         nonceGroup,
         nonce,
-        mvcc);
+        mvcc, null);
   }
 
   @InterfaceAudience.Private
@@ -339,7 +408,8 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
                       List<UUID> clusterIds,
                       long nonceGroup,
                       long nonce,
-                      MultiVersionConcurrencyControl mvcc) {
+                      MultiVersionConcurrencyControl mvcc,
+                      NavigableMap<byte[], Integer> replicationScope) {
     this.sequenceId = logSeqNum;
     this.writeTime = now;
     this.clusterIds = clusterIds;
@@ -351,6 +421,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     if (logSeqNum != NO_SEQUENCE_ID) {
       setSequenceId(logSeqNum);
     }
+    this.replicationScope = replicationScope;
   }
 
   // For HLogKey and deserialization. DO NOT USE. See setWriteEntry below.
@@ -418,8 +489,8 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     return this.writeTime;
   }
 
-  public NavigableMap<byte[], Integer> getScopes() {
-    return scopes;
+  public NavigableMap<byte[], Integer> getReplicationScopes() {
+    return replicationScope;
   }
 
   /** @return The nonce group */
@@ -432,8 +503,14 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     return nonce;
   }
 
-  public void setScopes(NavigableMap<byte[], Integer> scopes) {
-    this.scopes = scopes;
+  private void setReplicationScope(NavigableMap<byte[], Integer> replicationScope) {
+    this.replicationScope = replicationScope;
+  }
+
+  public void serializeReplicationScope(boolean serialize) {
+    if (!serialize) {
+      setReplicationScope(null);
+    }
   }
 
   public void readOlderScopes(NavigableMap<byte[], Integer> scopes) {
@@ -450,7 +527,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
         }
       }
       if (scopes.size() > 0) {
-        this.scopes = scopes;
+        this.replicationScope = scopes;
       }
     }
   }
@@ -598,8 +675,8 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
       uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
       builder.addClusterIds(uuidBuilder.build());
     }
-    if (scopes != null) {
-      for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
+    if (replicationScope != null) {
+      for (Map.Entry<byte[], Integer> e : replicationScope.entrySet()) {
         ByteString family = (compressionContext == null) ? ByteStringer.wrap(e.getKey())
             : compressor.compress(e.getKey(), compressionContext.familyDict);
         builder.addScopes(FamilyScope.newBuilder()
@@ -638,13 +715,13 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     if (walKey.hasNonce()) {
       this.nonce = walKey.getNonce();
     }
-    this.scopes = null;
+    this.replicationScope = null;
     if (walKey.getScopesCount() > 0) {
-      this.scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+      this.replicationScope = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
       for (FamilyScope scope : walKey.getScopesList()) {
         byte[] family = (compressionContext == null) ? scope.getFamily().toByteArray() :
           uncompressor.uncompress(scope.getFamily(), compressionContext.familyDict);
-        this.scopes.put(family, scope.getScopeType().getNumber());
+        this.replicationScope.put(family, scope.getScopeType().getNumber());
       }
     }
     setSequenceId(walKey.getLogSequenceNumber());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 010fd37..ad5774f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -1778,7 +1778,7 @@ public class WALSplitter {
         WALEdit edit = entry.getEdit();
         TableName table = entry.getKey().getTablename();
         // clear scopes which isn't needed for recovery
-        entry.getKey().setScopes(null);
+        entry.getKey().serializeReplicationScope(false);
         String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName());
         // skip edits of non-existent tables
         if (nonExistentTables != null && nonExistentTables.contains(table)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index 3aae5d5..d8363d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -281,7 +281,8 @@ public class TestIOFencing {
       CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(oldHri,
         FAMILY, Lists.newArrayList(new Path("/a")), Lists.newArrayList(new Path("/b")),
         new Path("store_dir"));
-      WALUtil.writeCompactionMarker(compactingRegion.getWAL(), table.getTableDescriptor(),
+      WALUtil.writeCompactionMarker(compactingRegion.getWAL(),
+          ((HRegion)compactingRegion).getReplicationScope(),
         oldHri, compactionDescriptor, compactingRegion.getMVCC());
 
       // Wait till flush has happened, otherwise there won't be multiple store files

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
index 75fe7a2..03760d1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
@@ -29,6 +29,8 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -184,7 +186,11 @@ public class TestWALObserver {
     HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
     final HTableDescriptor htd = createBasic3FamilyHTD(Bytes
         .toString(TEST_TABLE));
-
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     Path basedir = new Path(this.hbaseRootDir, Bytes.toString(TEST_TABLE));
     deleteDir(basedir);
     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
@@ -235,8 +241,8 @@ public class TestWALObserver {
     // it's where WAL write cp should occur.
     long now = EnvironmentEdgeManager.currentTime();
     // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
-    long txid = log.append(htd, hri, new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), now),
-        edit, true);
+    long txid = log.append(hri,
+        new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, scopes), edit, true);
     log.sync(txid);
 
     // the edit shall have been change now by the coprocessor.
@@ -296,10 +302,15 @@ public class TestWALObserver {
     assertFalse(oldApi.isPostWALWriteDeprecatedCalled());
 
     LOG.debug("writing to WAL with non-legacy keys.");
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for (HColumnDescriptor hcd : htd.getFamilies()) {
+      scopes.put(hcd.getName(), 0);
+    }
     final int countPerFamily = 5;
     for (HColumnDescriptor hcd : htd.getFamilies()) {
       addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
-          EnvironmentEdgeManager.getDelegate(), wal, htd, mvcc);
+          EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc);
     }
 
     LOG.debug("Verify that only the non-legacy CP saw edits.");
@@ -323,7 +334,7 @@ public class TestWALObserver {
     final WALEdit edit = new WALEdit();
     final byte[] nonce = Bytes.toBytes("1772");
     edit.add(new KeyValue(TEST_ROW, TEST_FAMILY[0], nonce, now, nonce));
-    final long txid = wal.append(htd, hri, legacyKey, edit, true);
+    final long txid = wal.append(hri, legacyKey, edit, true);
     wal.sync(txid);
 
     LOG.debug("Make sure legacy cps can see supported edits after having been skipped.");
@@ -349,7 +360,11 @@ public class TestWALObserver {
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
     final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE));
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
-
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for(byte[] fam : htd.getFamiliesKeys()) {
+      scopes.put(fam, 0);
+    }
     WAL log = wals.getWAL(UNSPECIFIED_REGION, null);
     try {
       SampleRegionWALObserver cp = getCoprocessor(log, SampleRegionWALObserver.class);
@@ -360,8 +375,8 @@ public class TestWALObserver {
       assertFalse(cp.isPostWALWriteCalled());
 
       final long now = EnvironmentEdgeManager.currentTime();
-      long txid = log.append(htd, hri,
-          new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc),
+      long txid = log.append(hri,
+          new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc, scopes),
           new WALEdit(), true);
       log.sync(txid);
 
@@ -400,14 +415,18 @@ public class TestWALObserver {
     // Put p = creatPutWith2Families(TEST_ROW);
     WALEdit edit = new WALEdit();
     long now = EnvironmentEdgeManager.currentTime();
-    // addFamilyMapToWALEdit(p.getFamilyMap(), edit);
     final int countPerFamily = 1000;
-    // for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+        Bytes.BYTES_COMPARATOR);
+    for (HColumnDescriptor hcd : htd.getFamilies()) {
+      scopes.put(hcd.getName(), 0);
+    }
     for (HColumnDescriptor hcd : htd.getFamilies()) {
       addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
-          EnvironmentEdgeManager.getDelegate(), wal, htd, mvcc);
+          EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc);
     }
-    wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc), edit, true);
+    wal.append(hri, new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit,
+        true);
     // sync to fs.
     wal.sync();
 
@@ -527,7 +546,8 @@ public class TestWALObserver {
 
   private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName,
       final byte[] family, final int count, EnvironmentEdge ee, final WAL wal,
-      final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc) throws IOException {
+      final NavigableMap<byte[], Integer> scopes, final MultiVersionConcurrencyControl mvcc)
+          throws IOException {
     String familyStr = Bytes.toString(family);
     long txid = -1;
     for (int j = 0; j < count; j++) {
@@ -537,7 +557,7 @@ public class TestWALObserver {
       edit.add(new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes));
       // uses WALKey instead of HLogKey on purpose. will only work for tests where we don't care
       // about legacy coprocessors
-      txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
+      txid = wal.append(hri, new WALKey(hri.getEncodedNameAsBytes(), tableName,
           ee.currentTime(), mvcc), edit, true);
     }
     if (-1 != txid) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
index 5fa588b..752faa6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
@@ -18,6 +18,9 @@
 package org.apache.hadoop.hbase.mapreduce;
 
 import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader;
+
+import java.util.NavigableMap;
+
 import org.apache.hadoop.hbase.mapreduce.HLogInputFormat.HLogKeyRecordReader;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -32,8 +35,8 @@ import org.junit.experimental.categories.Category;
 public class TestHLogRecordReader extends TestWALRecordReader {
 
   @Override
-  protected WALKey getWalKey(final long time) {
-    return new HLogKey(info.getEncodedNameAsBytes(), tableName, time, mvcc);
+  protected WALKey getWalKey(final long time, NavigableMap<byte[], Integer> scopes) {
+    return new HLogKey(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
index 05f9f36..094fe1c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
@@ -34,6 +34,7 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.NavigableMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -656,9 +657,9 @@ public class TestImportExport {
       Table importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3);
 
       // Register the wal listener for the import table
-      TableWALActionListener walListener = new TableWALActionListener(importTableName);
       HRegionInfo region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
           .getOnlineRegions(importTable.getName()).get(0).getRegionInfo();
+      TableWALActionListener walListener = new TableWALActionListener(region);
       WAL wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
       wal.registerWALActionsListener(walListener);
 
@@ -678,7 +679,7 @@ public class TestImportExport {
       region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
           .getOnlineRegions(importTable.getName()).get(0).getRegionInfo();
       wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
-      walListener = new TableWALActionListener(importTableName);
+      walListener = new TableWALActionListener(region);
       wal.registerWALActionsListener(walListener);
       args = new String[] { importTableName, FQ_OUTPUT_DIR };
       assertTrue(runImport(args));
@@ -695,16 +696,17 @@ public class TestImportExport {
    */
   private static class TableWALActionListener extends WALActionsListener.Base {
 
-    private String tableName;
+    private HRegionInfo regionInfo;
     private boolean isVisited = false;
 
-    public TableWALActionListener(String tableName) {
-      this.tableName = tableName;
+    public TableWALActionListener(HRegionInfo region) {
+      this.regionInfo = region;
     }
 
     @Override
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {
-      if (tableName.equalsIgnoreCase(htd.getNameAsString()) && (!logEdit.isMetaEdit())) {
+    public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
+      if (logKey.getTablename().getNameAsString().equalsIgnoreCase(
+          this.regionInfo.getTable().getNameAsString()) && (!logEdit.isMetaEdit())) {
         isVisited = true;
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
index a4381c8..aee2a06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
@@ -77,6 +79,8 @@ public class TestWALRecordReader {
   private static HTableDescriptor htd;
   private static Path logDir;
   protected MultiVersionConcurrencyControl mvcc;
+  protected static NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
+      Bytes.BYTES_COMPARATOR);
 
   private static String getName() {
     return "TestWALRecordReader";
@@ -128,10 +132,10 @@ public class TestWALRecordReader {
     long ts = System.currentTimeMillis();
     WALEdit edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value));
-    log.append(htd, info, getWalKey(ts), edit, true);
+    log.append(info, getWalKey(ts, scopes), edit, true);
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value));
-    log.append(htd, info, getWalKey(ts+1), edit, true);
+    log.append(info, getWalKey(ts+1, scopes), edit, true);
     log.sync();
     LOG.info("Before 1st WAL roll " + log.toString());
     log.rollWriter();
@@ -142,10 +146,10 @@ public class TestWALRecordReader {
 
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value));
-    log.append(htd, info, getWalKey(ts1+1), edit, true);
+    log.append(info, getWalKey(ts1+1, scopes), edit, true);
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value));
-    log.append(htd, info, getWalKey(ts1+2), edit, true);
+    log.append(info, getWalKey(ts1+2, scopes), edit, true);
     log.sync();
     log.shutdown();
     walfactory.shutdown();
@@ -187,7 +191,7 @@ public class TestWALRecordReader {
     WALEdit edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
         System.currentTimeMillis(), value));
-    long txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, true);
+    long txid = log.append(info, getWalKey(System.currentTimeMillis(), scopes), edit, true);
     log.sync(txid);
 
     Thread.sleep(1); // make sure 2nd log gets a later timestamp
@@ -197,7 +201,7 @@ public class TestWALRecordReader {
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
         System.currentTimeMillis(), value));
-    txid = log.append(htd, info, getWalKey(System.currentTimeMillis()), edit, true);
+    txid = log.append(info, getWalKey(System.currentTimeMillis(), scopes), edit, true);
     log.sync(txid);
     log.shutdown();
     walfactory.shutdown();
@@ -236,8 +240,8 @@ public class TestWALRecordReader {
     testSplit(splits.get(1));
   }
 
-  protected WALKey getWalKey(final long time) {
-    return new WALKey(info.getEncodedNameAsBytes(), tableName, time, mvcc);
+  protected WALKey getWalKey(final long time, NavigableMap<byte[], Integer> scopes) {
+    return new WALKey(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes);
   }
 
   protected WALRecordReader getReader() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index c5728cf..cff8db0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -1301,9 +1301,8 @@ public class TestDistributedLogSplitting {
         WALEdit e = new WALEdit();
         value++;
         e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
-        wal.append(htd, curRegionInfo,
-            new HLogKey(curRegionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis()),
-            e, true);
+        wal.append(curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(), tableName,
+            System.currentTimeMillis(), null), e, true);
       }
       wal.sync();
       wal.shutdown();
@@ -1397,7 +1396,7 @@ public class TestDistributedLogSplitting {
         WALEdit e = new WALEdit();
         value++;
         e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
-        wal.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(),
+        wal.append(curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(),
             tableName, System.currentTimeMillis()), e, true);
       }
       wal.sync();
@@ -1609,7 +1608,7 @@ public class TestDistributedLogSplitting {
         // key
         byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
         e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
-        log.append(htd, curRegionInfo,
+        log.append(curRegionInfo,
             new HLogKey(curRegionInfo.getEncodedNameAsBytes(), fullTName,
                 System.currentTimeMillis()), e, true);
         if (0 == i % syncEvery) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
index 3a7aff0..d0633a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -87,6 +87,7 @@ public class TestBulkLoad {
   private final byte[] randomBytes = new byte[100];
   private final byte[] family1 = Bytes.toBytes("family1");
   private final byte[] family2 = Bytes.toBytes("family2");
+
   @Rule
   public TestName name = new TestName();
 
@@ -105,12 +106,12 @@ public class TestBulkLoad {
     storeFileName = (new Path(storeFileName)).getName();
     List<String> storeFileNames = new ArrayList<String>();
     storeFileNames.add(storeFileName);
-    when(log.append(any(HTableDescriptor.class), any(HRegionInfo.class), any(WALKey.class),
+    when(log.append(any(HRegionInfo.class), any(WALKey.class),
             argThat(bulkLogWalEdit(WALEdit.BULK_LOAD, tableName.toBytes(),
                     familyName, storeFileNames)),
             any(boolean.class))).thenAnswer(new Answer() {
       public Object answer(InvocationOnMock invocation) {
-        WALKey walKey = invocation.getArgumentAt(2, WALKey.class);
+        WALKey walKey = invocation.getArgumentAt(1, WALKey.class);
         MultiVersionConcurrencyControl mvcc = walKey.getMvcc();
         if (mvcc != null) {
           MultiVersionConcurrencyControl.WriteEntry we = mvcc.begin();
@@ -132,11 +133,11 @@ public class TestBulkLoad {
 
   @Test
   public void shouldBulkLoadSingleFamilyHLog() throws IOException {
-    when(log.append(any(HTableDescriptor.class), any(HRegionInfo.class),
+    when(log.append(any(HRegionInfo.class),
             any(WALKey.class), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)),
             any(boolean.class))).thenAnswer(new Answer() {
       public Object answer(InvocationOnMock invocation) {
-        WALKey walKey = invocation.getArgumentAt(2, WALKey.class);
+        WALKey walKey = invocation.getArgumentAt(1, WALKey.class);
         MultiVersionConcurrencyControl mvcc = walKey.getMvcc();
         if (mvcc != null) {
           MultiVersionConcurrencyControl.WriteEntry we = mvcc.begin();
@@ -151,11 +152,11 @@ public class TestBulkLoad {
 
   @Test
   public void shouldBulkLoadManyFamilyHLog() throws IOException {
-    when(log.append(any(HTableDescriptor.class), any(HRegionInfo.class),
+    when(log.append(any(HRegionInfo.class),
             any(WALKey.class), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)),
             any(boolean.class))).thenAnswer(new Answer() {
               public Object answer(InvocationOnMock invocation) {
-                WALKey walKey = invocation.getArgumentAt(2, WALKey.class);
+                WALKey walKey = invocation.getArgumentAt(1, WALKey.class);
                 MultiVersionConcurrencyControl mvcc = walKey.getMvcc();
                 if (mvcc != null) {
                   MultiVersionConcurrencyControl.WriteEntry we = mvcc.begin();
@@ -171,11 +172,11 @@ public class TestBulkLoad {
 
   @Test
   public void shouldBulkLoadManyFamilyHLogEvenWhenTableNameNamespaceSpecified() throws IOException {
-    when(log.append(any(HTableDescriptor.class), any(HRegionInfo.class),
+    when(log.append(any(HRegionInfo.class),
             any(WALKey.class), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)),
             any(boolean.class))).thenAnswer(new Answer() {
       public Object answer(InvocationOnMock invocation) {
-        WALKey walKey = invocation.getArgumentAt(2, WALKey.class);
+        WALKey walKey = invocation.getArgumentAt(1, WALKey.class);
         MultiVersionConcurrencyControl mvcc = walKey.getMvcc();
         if (mvcc != null) {
           MultiVersionConcurrencyControl.WriteEntry we = mvcc.begin();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/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 a5574d3..ed7623c 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
@@ -896,7 +896,7 @@ public class TestHRegion {
             storeFiles, Lists.newArrayList(newFile),
             region.getRegionFileSystem().getStoreDir(Bytes.toString(family)));
 
-      WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(),
+      WALUtil.writeCompactionMarker(region.getWAL(), this.region.getReplicationScope(),
           this.region.getRegionInfo(), compactionDescriptor, region.getMVCC());
 
       Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
@@ -4796,7 +4796,7 @@ public class TestHRegion {
 
     //verify append called or not
     verify(wal, expectAppend ? times(1) : never())
-      .append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(),
+      .append((HRegionInfo)any(), (WALKey)any(),
           (WALEdit)any(), Mockito.anyBoolean());
 
     // verify sync called or not
@@ -5998,7 +5998,7 @@ public class TestHRegion {
       region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
         TEST_UTIL.getConfiguration(), rss, null);
 
-      verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
+      verify(wal, times(1)).append((HRegionInfo)any(), (WALKey)any()
         , editCaptor.capture(), anyBoolean());
 
       WALEdit edit = editCaptor.getValue();
@@ -6111,7 +6111,7 @@ public class TestHRegion {
 
       // verify that we have not appended region open event to WAL because this region is still
       // recovering
-      verify(wal, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
+      verify(wal, times(0)).append((HRegionInfo)any(), (WALKey)any()
         , editCaptor.capture(), anyBoolean());
 
       // not put the region out of recovering state
@@ -6119,7 +6119,7 @@ public class TestHRegion {
         .prepare().process();
 
       // now we should have put the entry
-      verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
+      verify(wal, times(1)).append((HRegionInfo)any(), (WALKey)any()
         , editCaptor.capture(), anyBoolean());
 
       WALEdit edit = editCaptor.getValue();
@@ -6163,12 +6163,12 @@ public class TestHRegion {
    */
   private WAL mockWAL() throws IOException {
     WAL wal = mock(WAL.class);
-    Mockito.when(wal.append((HTableDescriptor)Mockito.any(), (HRegionInfo)Mockito.any(),
+    Mockito.when(wal.append((HRegionInfo)Mockito.any(),
         (WALKey)Mockito.any(), (WALEdit)Mockito.any(), Mockito.anyBoolean())).
       thenAnswer(new Answer<Long>() {
         @Override
         public Long answer(InvocationOnMock invocation) throws Throwable {
-          WALKey key = invocation.getArgumentAt(2, WALKey.class);
+          WALKey key = invocation.getArgumentAt(1, WALKey.class);
           MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin();
           key.setWriteEntry(we);
           return 1L;
@@ -6206,7 +6206,7 @@ public class TestHRegion {
     region.close(false);
 
     // 2 times, one for region open, the other close region
-    verify(wal, times(2)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(),
+    verify(wal, times(2)).append((HRegionInfo)any(), (WALKey)any(),
       editCaptor.capture(), anyBoolean());
 
     WALEdit edit = editCaptor.getAllValues().get(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 4d5d7d8..9183e18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -1126,7 +1126,7 @@ public class TestHRegionReplayEvents {
 
     // test for region open and close
     secondaryRegion = HRegion.openHRegion(secondaryHri, htd, walSecondary, CONF, rss, null);
-    verify(walSecondary, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(),
+    verify(walSecondary, times(0)).append((HRegionInfo)any(),
       (WALKey)any(), (WALEdit)any(),  anyBoolean());
 
     // test for replay prepare flush
@@ -1140,11 +1140,11 @@ public class TestHRegionReplayEvents {
       .setRegionName(ByteString.copyFrom(primaryRegion.getRegionInfo().getRegionName()))
       .build());
 
-    verify(walSecondary, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(),
+    verify(walSecondary, times(0)).append((HRegionInfo)any(),
       (WALKey)any(), (WALEdit)any(), anyBoolean());
 
     secondaryRegion.close();
-    verify(walSecondary, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(),
+    verify(walSecondary, times(0)).append((HRegionInfo)any(),
       (WALKey)any(), (WALEdit)any(),  anyBoolean());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f2bd060/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 87cbab7..76b4134 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
@@ -411,7 +412,7 @@ public class TestHRegionServerBulkLoad {
     private boolean found = false;
 
     @Override
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {
+    public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
       for (Cell cell : logEdit.getCells()) {
         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
         for (Map.Entry entry : kv.toStringMap().entrySet()) {


[14/37] hbase git commit: HBASE-15310 hbase-spark module has compilation failures with clover profile

Posted by sy...@apache.org.
HBASE-15310 hbase-spark module has compilation failures with clover profile


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

Branch: refs/heads/hbase-12439
Commit: 20e14f449a9d5ba052ef6250c08ee1e4c558ccf2
Parents: 2a30643
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Wed Feb 24 10:09:21 2016 -0800
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Wed Feb 24 11:54:43 2016 -0800

----------------------------------------------------------------------
 hbase-spark/pom.xml | 35 +++++++++++++++++++++++++++++++++++
 1 file changed, 35 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/20e14f44/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 7c7590e..7767440 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -604,6 +604,41 @@
                     </execution>
                 </executions>
             </plugin>
+
+            <!-- clover fails due to scala/java cross compile.  This guarantees that the scala is
+             compiled before the java that will be evaluated by code coverage (scala will not be).
+            https://confluence.atlassian.com/display/CLOVERKB/Java-+Scala+cross-compilation+error+-+cannot+find+symbol
+            -->
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>validate</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>add-test-source</id>
+                        <phase>validate</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/scala</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 


[18/37] hbase git commit: HBASE-15264 Implement a fan out HDFS OutputStream

Posted by sy...@apache.org.
HBASE-15264 Implement a fan out HDFS OutputStream


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

Branch: refs/heads/hbase-12439
Commit: 6e9d355b12a1e666f4d05be02775a01b6754d063
Parents: a3b4575
Author: zhangduo <zh...@apache.org>
Authored: Wed Feb 24 20:47:38 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Feb 25 10:07:27 2016 +0800

----------------------------------------------------------------------
 .../util/FanOutOneBlockAsyncDFSOutput.java      | 533 +++++++++++++++
 .../FanOutOneBlockAsyncDFSOutputHelper.java     | 672 +++++++++++++++++++
 ...anOutOneBlockAsyncDFSOutputFlushHandler.java |  61 ++
 .../util/TestFanOutOneBlockAsyncDFSOutput.java  | 190 ++++++
 4 files changed, 1456 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6e9d355b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutput.java
new file mode 100644
index 0000000..b10f180
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutput.java
@@ -0,0 +1,533 @@
+/**
+ * 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.util;
+
+import static org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.HEART_BEAT_SEQNO;
+import static org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.completeFile;
+import static org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.endFileLease;
+import static org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.getStatus;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.channels.CompletionHandler;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.FanOutOneBlockAsyncDFSOutputHelper.CancelOnClose;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.util.DataChecksum;
+
+import com.google.common.base.Supplier;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.EventLoop;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.handler.codec.protobuf.ProtobufDecoder;
+import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import io.netty.handler.timeout.IdleState;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.FutureListener;
+import io.netty.util.concurrent.Promise;
+
+/**
+ * An asynchronous HDFS output stream implementation which fans out data to datanode and only
+ * supports writing file with only one block.
+ * <p>
+ * Use the createOutput method in {@link FanOutOneBlockAsyncDFSOutputHelper} to create. The mainly
+ * usage of this class is implementing WAL, so we only expose a little HDFS configurations in the
+ * method. And we place it here under util package because we want to make it independent of WAL
+ * implementation thus easier to move it to HDFS project finally.
+ * <p>
+ * Note that, all connections to datanode will run in the same {@link EventLoop} which means we only
+ * need one thread here. But be careful, we do some blocking operations in {@link #close()} and
+ * {@link #recoverAndClose(CancelableProgressable)} methods, so do not call them inside
+ * {@link EventLoop}. And for {@link #write(byte[])} {@link #write(byte[], int, int)},
+ * {@link #buffered()} and {@link #flush(Object, CompletionHandler, boolean)}, if you call them
+ * outside {@link EventLoop}, there will be an extra context-switch.
+ * <p>
+ * Advantages compare to DFSOutputStream:
+ * <ol>
+ * <li>The fan out mechanism. This will reduce the latency.</li>
+ * <li>The asynchronous WAL could also run in the same EventLoop, we could just call write and flush
+ * inside the EventLoop thread, so generally we only have one thread to do all the things.</li>
+ * <li>Fail-fast when connection to datanode error. The WAL implementation could open new writer
+ * ASAP.</li>
+ * <li>We could benefit from netty's ByteBuf management mechanism.</li>
+ * </ol>
+ */
+@InterfaceAudience.Private
+public class FanOutOneBlockAsyncDFSOutput implements Closeable {
+
+  private final Configuration conf;
+
+  private final FSUtils fsUtils;
+
+  private final DistributedFileSystem dfs;
+
+  private final DFSClient client;
+
+  private final ClientProtocol namenode;
+
+  private final String clientName;
+
+  private final String src;
+
+  private final long fileId;
+
+  private final LocatedBlock locatedBlock;
+
+  private final EventLoop eventLoop;
+
+  private final List<Channel> datanodeList;
+
+  private final DataChecksum summer;
+
+  private final ByteBufAllocator alloc;
+
+  private static final class Callback {
+
+    public final Promise<Void> promise;
+
+    public final long ackedLength;
+
+    public final Set<Channel> unfinishedReplicas;
+
+    public Callback(Promise<Void> promise, long ackedLength, Collection<Channel> replicas) {
+      this.promise = promise;
+      this.ackedLength = ackedLength;
+      if (replicas.isEmpty()) {
+        this.unfinishedReplicas = Collections.emptySet();
+      } else {
+        this.unfinishedReplicas = Collections
+            .newSetFromMap(new IdentityHashMap<Channel, Boolean>(replicas.size()));
+        this.unfinishedReplicas.addAll(replicas);
+      }
+    }
+  }
+
+  private final Deque<Callback> waitingAckQueue = new ArrayDeque<>();
+
+  // this could be different from acked block length because a packet can not start at the middle of
+  // a chunk.
+  private long nextPacketOffsetInBlock = 0L;
+
+  private long nextPacketSeqno = 0L;
+
+  private ByteBuf buf;
+
+  private enum State {
+    STREAMING, CLOSING, BROKEN, CLOSED
+  }
+
+  private State state;
+
+  private void completed(Channel channel) {
+    if (waitingAckQueue.isEmpty()) {
+      return;
+    }
+    for (Callback c : waitingAckQueue) {
+      if (c.unfinishedReplicas.remove(channel)) {
+        if (c.unfinishedReplicas.isEmpty()) {
+          c.promise.trySuccess(null);
+          // since we will remove the Callback entry from waitingAckQueue if its unfinishedReplicas
+          // is empty, so this could only happen at the head of waitingAckQueue, so we just call
+          // removeFirst here.
+          waitingAckQueue.removeFirst();
+          // also wake up flush requests which have the same length.
+          for (Callback cb; (cb = waitingAckQueue.peekFirst()) != null;) {
+            if (cb.ackedLength == c.ackedLength) {
+              cb.promise.trySuccess(null);
+              waitingAckQueue.removeFirst();
+            } else {
+              break;
+            }
+          }
+        }
+        return;
+      }
+    }
+  }
+
+  private void failed(Channel channel, Supplier<Throwable> errorSupplier) {
+    if (state == State.BROKEN || state == State.CLOSED) {
+      return;
+    }
+    if (state == State.CLOSING) {
+      Callback c = waitingAckQueue.peekFirst();
+      if (c == null || !c.unfinishedReplicas.contains(channel)) {
+        // nothing, the endBlock request has already finished.
+        return;
+      }
+    }
+    // disable further write, and fail all pending ack.
+    state = State.BROKEN;
+    Throwable error = errorSupplier.get();
+    for (Callback c : waitingAckQueue) {
+      c.promise.tryFailure(error);
+    }
+    waitingAckQueue.clear();
+    for (Channel ch : datanodeList) {
+      ch.close();
+    }
+  }
+
+  private void setupReceiver(final int timeoutMs) {
+    SimpleChannelInboundHandler<PipelineAckProto> ackHandler = new SimpleChannelInboundHandler<PipelineAckProto>() {
+
+      @Override
+      public boolean isSharable() {
+        return true;
+      }
+
+      @Override
+      protected void channelRead0(final ChannelHandlerContext ctx, PipelineAckProto ack)
+          throws Exception {
+        final Status reply = getStatus(ack);
+        if (reply != Status.SUCCESS) {
+          failed(ctx.channel(), new Supplier<Throwable>() {
+
+            @Override
+            public Throwable get() {
+              return new IOException("Bad response " + reply + " for block "
+                  + locatedBlock.getBlock() + " from datanode " + ctx.channel().remoteAddress());
+            }
+          });
+          return;
+        }
+        if (PipelineAck.isRestartOOBStatus(reply)) {
+          failed(ctx.channel(), new Supplier<Throwable>() {
+
+            @Override
+            public Throwable get() {
+              return new IOException("Restart response " + reply + " for block "
+                  + locatedBlock.getBlock() + " from datanode " + ctx.channel().remoteAddress());
+            }
+          });
+          return;
+        }
+        if (ack.getSeqno() == HEART_BEAT_SEQNO) {
+          return;
+        }
+        completed(ctx.channel());
+      }
+
+      @Override
+      public void channelInactive(final ChannelHandlerContext ctx) throws Exception {
+        failed(ctx.channel(), new Supplier<Throwable>() {
+
+          @Override
+          public Throwable get() {
+            return new IOException("Connection to " + ctx.channel().remoteAddress() + " closed");
+          }
+        });
+      }
+
+      @Override
+      public void exceptionCaught(ChannelHandlerContext ctx, final Throwable cause)
+          throws Exception {
+        failed(ctx.channel(), new Supplier<Throwable>() {
+
+          @Override
+          public Throwable get() {
+            return cause;
+          }
+        });
+      }
+
+      @Override
+      public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
+        if (evt instanceof IdleStateEvent) {
+          IdleStateEvent e = (IdleStateEvent) evt;
+          if (e.state() == IdleState.READER_IDLE) {
+            failed(ctx.channel(), new Supplier<Throwable>() {
+
+              @Override
+              public Throwable get() {
+                return new IOException("Timeout(" + timeoutMs + "ms) waiting for response");
+              }
+            });
+          } else if (e.state() == IdleState.WRITER_IDLE) {
+            PacketHeader heartbeat = new PacketHeader(4, 0, HEART_BEAT_SEQNO, false, 0, false);
+            int len = heartbeat.getSerializedSize();
+            ByteBuf buf = alloc.buffer(len);
+            heartbeat.putInBuffer(buf.nioBuffer(0, len));
+            buf.writerIndex(len);
+            ctx.channel().writeAndFlush(buf);
+          }
+          return;
+        }
+        super.userEventTriggered(ctx, evt);
+      }
+
+    };
+    for (Channel ch : datanodeList) {
+      ch.pipeline().addLast(
+        new IdleStateHandler(timeoutMs, timeoutMs / 2, 0, TimeUnit.MILLISECONDS),
+        new ProtobufVarint32FrameDecoder(),
+        new ProtobufDecoder(PipelineAckProto.getDefaultInstance()), ackHandler);
+      ch.config().setAutoRead(true);
+    }
+  }
+
+  FanOutOneBlockAsyncDFSOutput(Configuration conf, FSUtils fsUtils, DistributedFileSystem dfs,
+      DFSClient client, ClientProtocol namenode, String clientName, String src, long fileId,
+      LocatedBlock locatedBlock, EventLoop eventLoop, List<Channel> datanodeList,
+      DataChecksum summer, ByteBufAllocator alloc) {
+    this.conf = conf;
+    this.fsUtils = fsUtils;
+    this.dfs = dfs;
+    this.client = client;
+    this.namenode = namenode;
+    this.fileId = fileId;
+    this.clientName = clientName;
+    this.src = src;
+    this.locatedBlock = locatedBlock;
+    this.eventLoop = eventLoop;
+    this.datanodeList = datanodeList;
+    this.summer = summer;
+    this.alloc = alloc;
+    this.buf = alloc.directBuffer();
+    this.state = State.STREAMING;
+    setupReceiver(conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, HdfsServerConstants.READ_TIMEOUT));
+  }
+
+  /**
+   * Just call write(b, 0, b.length).
+   * @see #write(byte[], int, int)
+   */
+  public void write(byte[] b) {
+    write(b, 0, b.length);
+  }
+
+  /**
+   * Copy the data into the buffer. Note that you need to call
+   * {@link #flush(Object, CompletionHandler, boolean)} to flush the buffer manually.
+   */
+  public void write(final byte[] b, final int off, final int len) {
+    if (eventLoop.inEventLoop()) {
+      buf.ensureWritable(len).writeBytes(b, off, len);
+    } else {
+      eventLoop.submit(new Runnable() {
+
+        @Override
+        public void run() {
+          buf.ensureWritable(len).writeBytes(b, off, len);
+        }
+      }).syncUninterruptibly();
+    }
+  }
+
+  /**
+   * Return the current size of buffered data.
+   */
+  public int buffered() {
+    if (eventLoop.inEventLoop()) {
+      return buf.readableBytes();
+    } else {
+      return eventLoop.submit(new Callable<Integer>() {
+
+        @Override
+        public Integer call() throws Exception {
+          return buf.readableBytes();
+        }
+      }).syncUninterruptibly().getNow().intValue();
+    }
+  }
+
+  public DatanodeInfo[] getPipeline() {
+    return locatedBlock.getLocations();
+  }
+
+  private <A> void flush0(final A attachment, final CompletionHandler<Long, ? super A> handler,
+      boolean syncBlock) {
+    if (state != State.STREAMING) {
+      handler.failed(new IOException("stream already broken"), attachment);
+      return;
+    }
+    int dataLen = buf.readableBytes();
+    final long ackedLength = nextPacketOffsetInBlock + dataLen;
+    if (ackedLength == locatedBlock.getBlock().getNumBytes()) {
+      // no new data, just return
+      handler.completed(locatedBlock.getBlock().getNumBytes(), attachment);
+      return;
+    }
+    Promise<Void> promise = eventLoop.newPromise();
+    promise.addListener(new FutureListener<Void>() {
+
+      @Override
+      public void operationComplete(Future<Void> future) throws Exception {
+        if (future.isSuccess()) {
+          locatedBlock.getBlock().setNumBytes(ackedLength);
+          handler.completed(ackedLength, attachment);
+        } else {
+          handler.failed(future.cause(), attachment);
+        }
+      }
+    });
+    Callback c = waitingAckQueue.peekLast();
+    if (c != null && ackedLength == c.ackedLength) {
+      // just append it to the tail of waiting ack queue,, do not issue new hflush request.
+      waitingAckQueue
+          .addLast(new Callback(promise, ackedLength, Collections.<Channel> emptyList()));
+      return;
+    }
+    int chunkLen = summer.getBytesPerChecksum();
+    int trailingPartialChunkLen = dataLen % chunkLen;
+    int numChecks = dataLen / chunkLen + (trailingPartialChunkLen != 0 ? 1 : 0);
+    int checksumLen = numChecks * summer.getChecksumSize();
+    ByteBuf checksumBuf = alloc.directBuffer(checksumLen);
+    summer.calculateChunkedSums(buf.nioBuffer(), checksumBuf.nioBuffer(0, checksumLen));
+    checksumBuf.writerIndex(checksumLen);
+    PacketHeader header = new PacketHeader(4 + checksumLen + dataLen, nextPacketOffsetInBlock,
+        nextPacketSeqno, false, dataLen, syncBlock);
+    int headerLen = header.getSerializedSize();
+    ByteBuf headerBuf = alloc.buffer(headerLen);
+    header.putInBuffer(headerBuf.nioBuffer(0, headerLen));
+    headerBuf.writerIndex(headerLen);
+
+    waitingAckQueue.addLast(new Callback(promise, ackedLength, datanodeList));
+    for (Channel ch : datanodeList) {
+      ch.write(headerBuf.duplicate().retain());
+      ch.write(checksumBuf.duplicate().retain());
+      ch.writeAndFlush(buf.duplicate().retain());
+    }
+    checksumBuf.release();
+    headerBuf.release();
+    ByteBuf newBuf = alloc.directBuffer().ensureWritable(trailingPartialChunkLen);
+    if (trailingPartialChunkLen != 0) {
+      buf.readerIndex(dataLen - trailingPartialChunkLen).readBytes(newBuf, trailingPartialChunkLen);
+    }
+    buf.release();
+    this.buf = newBuf;
+    nextPacketOffsetInBlock += dataLen - trailingPartialChunkLen;
+    nextPacketSeqno++;
+  }
+
+  /**
+   * Flush the buffer out to datanodes.
+   * @param attachment will be passed to handler when completed.
+   * @param handler will set the acked length as result when completed.
+   * @param syncBlock will call hsync if true, otherwise hflush.
+   */
+  public <A> void flush(final A attachment, final CompletionHandler<Long, ? super A> handler,
+      final boolean syncBlock) {
+    if (eventLoop.inEventLoop()) {
+      flush0(attachment, handler, syncBlock);
+    } else {
+      eventLoop.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          flush0(attachment, handler, syncBlock);
+        }
+      });
+    }
+  }
+
+  private void endBlock(Promise<Void> promise, long size) {
+    if (state != State.STREAMING) {
+      promise.tryFailure(new IOException("stream already broken"));
+      return;
+    }
+    if (!waitingAckQueue.isEmpty()) {
+      promise.tryFailure(new IllegalStateException("should call flush first before calling close"));
+      return;
+    }
+    state = State.CLOSING;
+    PacketHeader header = new PacketHeader(4, size, nextPacketSeqno, true, 0, false);
+    buf.release();
+    buf = null;
+    int headerLen = header.getSerializedSize();
+    ByteBuf headerBuf = alloc.buffer(headerLen);
+    header.putInBuffer(headerBuf.nioBuffer(0, headerLen));
+    headerBuf.writerIndex(headerLen);
+    waitingAckQueue.add(new Callback(promise, size, datanodeList));
+    for (Channel ch : datanodeList) {
+      ch.writeAndFlush(headerBuf.duplicate().retain());
+    }
+    headerBuf.release();
+  }
+
+  /**
+   * The close method when error occurred. Now we just call recoverFileLease.
+   */
+  public void recoverAndClose(CancelableProgressable reporter) throws IOException {
+    assert !eventLoop.inEventLoop();
+    for (Channel ch : datanodeList) {
+      ch.closeFuture().awaitUninterruptibly();
+    }
+    endFileLease(client, src, fileId);
+    fsUtils.recoverFileLease(dfs, new Path(src), conf,
+      reporter == null ? new CancelOnClose(client) : reporter);
+  }
+
+  /**
+   * End the current block and complete file at namenode. You should call
+   * {@link #recoverAndClose(CancelableProgressable)} if this method throws an exception.
+   */
+  @Override
+  public void close() throws IOException {
+    assert !eventLoop.inEventLoop();
+    final Promise<Void> promise = eventLoop.newPromise();
+    eventLoop.execute(new Runnable() {
+
+      @Override
+      public void run() {
+        endBlock(promise, nextPacketOffsetInBlock + buf.readableBytes());
+      }
+    });
+    promise.addListener(new FutureListener<Void>() {
+
+      @Override
+      public void operationComplete(Future<Void> future) throws Exception {
+        for (Channel ch : datanodeList) {
+          ch.close();
+        }
+      }
+    }).syncUninterruptibly();
+    for (Channel ch : datanodeList) {
+      ch.closeFuture().awaitUninterruptibly();
+    }
+    completeFile(client, namenode, src, clientName, locatedBlock.getBlock(), fileId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e9d355b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
new file mode 100644
index 0000000..d34bbb0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -0,0 +1,672 @@
+/**
+ * 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.util;
+
+import static io.netty.channel.ChannelOption.CONNECT_TIMEOUT_MILLIS;
+import static org.apache.hadoop.fs.CreateFlag.CREATE;
+import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
+import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_CREATE;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemLinkResolver;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.Builder;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.DataChecksum;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.CodedOutputStream;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufOutputStream;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoop;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.protobuf.ProtobufDecoder;
+import io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import io.netty.handler.timeout.IdleState;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.Promise;
+
+/**
+ * Helper class for implementing {@link FanOutOneBlockAsyncDFSOutput}.
+ */
+@InterfaceAudience.Private
+public class FanOutOneBlockAsyncDFSOutputHelper {
+
+  private static final Log LOG = LogFactory.getLog(FanOutOneBlockAsyncDFSOutputHelper.class);
+
+  private FanOutOneBlockAsyncDFSOutputHelper() {
+  }
+
+  // use pooled allocator for performance.
+  private static final ByteBufAllocator ALLOC = PooledByteBufAllocator.DEFAULT;
+
+  // copied from DFSPacket since it is package private.
+  public static final long HEART_BEAT_SEQNO = -1L;
+
+  // helper class for creating DataChecksum object.
+  private static final Method CREATE_CHECKSUM;
+
+  // helper class for getting Status from PipelineAckProto. In hadoop 2.6 or before, there is a
+  // getStatus method, and for hadoop 2.7 or after, the status is retrieved from flag. The flag may
+  // get from proto directly, or combined by the reply field of the proto and a ECN object. See
+  // createPipelineAckStatusGetter for more details.
+  private interface PipelineAckStatusGetter {
+    Status get(PipelineAckProto ack);
+  }
+
+  private static final PipelineAckStatusGetter PIPELINE_ACK_STATUS_GETTER;
+
+  // StorageType enum is added in hadoop 2.4, but it is moved to another package in hadoop 2.6 and
+  // the setter method in OpWriteBlockProto is also added in hadoop 2.6. So we need to skip the
+  // setStorageType call if it is hadoop 2.5 or before. See createStorageTypeSetter for more
+  // details.
+  private interface StorageTypeSetter {
+    OpWriteBlockProto.Builder set(OpWriteBlockProto.Builder builder, Enum<?> storageType);
+  }
+
+  private static final StorageTypeSetter STORAGE_TYPE_SETTER;
+
+  // helper class for calling create method on namenode. There is a supportedVersions parameter for
+  // hadoop 2.6 or after. See createFileCreater for more details.
+  private interface FileCreater {
+    HdfsFileStatus create(ClientProtocol namenode, String src, FsPermission masked,
+        String clientName, EnumSetWritable<CreateFlag> flag, boolean createParent,
+        short replication, long blockSize) throws IOException;
+  }
+
+  private static final FileCreater FILE_CREATER;
+
+  // helper class for add or remove lease from DFSClient. Hadoop 2.4 use src as the Map's key, and
+  // hadoop 2.5 or after use inodeId. See createLeaseManager for more details.
+  private interface LeaseManager {
+
+    void begin(DFSClient client, String src, long inodeId);
+
+    void end(DFSClient client, String src, long inodeId);
+  }
+
+  private static final LeaseManager LEASE_MANAGER;
+
+  // This is used to terminate a recoverFileLease call when FileSystem is already closed.
+  // isClientRunning is not public so we need to use reflection.
+  private interface DFSClientAdaptor {
+    boolean isClientRunning(DFSClient client);
+  }
+
+  private static final DFSClientAdaptor DFS_CLIENT_ADAPTOR;
+
+  private static DFSClientAdaptor createDFSClientAdaptor() {
+    try {
+      final Method method = DFSClient.class.getDeclaredMethod("isClientRunning");
+      method.setAccessible(true);
+      return new DFSClientAdaptor() {
+
+        @Override
+        public boolean isClientRunning(DFSClient client) {
+          try {
+            return (Boolean) method.invoke(client);
+          } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+    } catch (NoSuchMethodException e) {
+      throw new Error(e);
+    }
+  }
+
+  private static LeaseManager createLeaseManager() {
+    try {
+      final Method beginFileLeaseMethod = DFSClient.class.getDeclaredMethod("beginFileLease",
+        long.class, DFSOutputStream.class);
+      beginFileLeaseMethod.setAccessible(true);
+      final Method endFileLeaseMethod = DFSClient.class.getDeclaredMethod("endFileLease",
+        long.class);
+      endFileLeaseMethod.setAccessible(true);
+      return new LeaseManager() {
+
+        @Override
+        public void begin(DFSClient client, String src, long inodeId) {
+          try {
+            beginFileLeaseMethod.invoke(client, inodeId, null);
+          } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+          }
+        }
+
+        @Override
+        public void end(DFSClient client, String src, long inodeId) {
+          try {
+            endFileLeaseMethod.invoke(client, inodeId);
+          } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+    } catch (NoSuchMethodException e) {
+      LOG.warn("No inodeId related lease methods found, should be hadoop 2.4-", e);
+    }
+    try {
+      final Method beginFileLeaseMethod = DFSClient.class.getDeclaredMethod("beginFileLease",
+        String.class, DFSOutputStream.class);
+      beginFileLeaseMethod.setAccessible(true);
+      final Method endFileLeaseMethod = DFSClient.class.getDeclaredMethod("endFileLease",
+        String.class);
+      endFileLeaseMethod.setAccessible(true);
+      return new LeaseManager() {
+
+        @Override
+        public void begin(DFSClient client, String src, long inodeId) {
+          try {
+            beginFileLeaseMethod.invoke(client, src, null);
+          } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+          }
+        }
+
+        @Override
+        public void end(DFSClient client, String src, long inodeId) {
+          try {
+            endFileLeaseMethod.invoke(client, src);
+          } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+    } catch (NoSuchMethodException e) {
+      throw new Error(e);
+    }
+  }
+
+  private static PipelineAckStatusGetter createPipelineAckStatusGetter() {
+    try {
+      final Method getFlagListMethod = PipelineAckProto.class.getMethod("getFlagList");
+      @SuppressWarnings("rawtypes")
+      Class<? extends Enum> ecnClass;
+      try {
+        ecnClass = Class.forName("org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck$ECN")
+            .asSubclass(Enum.class);
+      } catch (ClassNotFoundException e) {
+        throw new Error(e);
+      }
+      @SuppressWarnings("unchecked")
+      final Enum<?> disabledECN = Enum.valueOf(ecnClass, "DISABLED");
+      final Method getReplyMethod = PipelineAckProto.class.getMethod("getReply", int.class);
+      final Method combineHeaderMethod = PipelineAck.class.getMethod("combineHeader", ecnClass,
+        Status.class);
+      final Method getStatusFromHeaderMethod = PipelineAck.class.getMethod("getStatusFromHeader",
+        int.class);
+      return new PipelineAckStatusGetter() {
+
+        @Override
+        public Status get(PipelineAckProto ack) {
+          try {
+            @SuppressWarnings("unchecked")
+            List<Integer> flagList = (List<Integer>) getFlagListMethod.invoke(ack);
+            Integer headerFlag;
+            if (flagList.isEmpty()) {
+              Status reply = (Status) getReplyMethod.invoke(ack, 0);
+              headerFlag = (Integer) combineHeaderMethod.invoke(null, disabledECN, reply);
+            } else {
+              headerFlag = flagList.get(0);
+            }
+            return (Status) getStatusFromHeaderMethod.invoke(null, headerFlag);
+          } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+    } catch (NoSuchMethodException e) {
+      LOG.warn("Can not get expected methods, should be hadoop 2.6-", e);
+    }
+    try {
+      final Method getStatusMethod = PipelineAckProto.class.getMethod("getStatus", int.class);
+      return new PipelineAckStatusGetter() {
+
+        @Override
+        public Status get(PipelineAckProto ack) {
+          try {
+            return (Status) getStatusMethod.invoke(ack, 0);
+          } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      };
+    } catch (NoSuchMethodException e) {
+      throw new Error(e);
+    }
+  }
+
+  private static StorageTypeSetter createStorageTypeSetter() {
+    final Method setStorageTypeMethod;
+    try {
+      setStorageTypeMethod = OpWriteBlockProto.Builder.class.getMethod("setStorageType",
+        StorageTypeProto.class);
+    } catch (NoSuchMethodException e) {
+      LOG.warn("noSetStorageType method found, should be hadoop 2.5-", e);
+      return new StorageTypeSetter() {
+
+        @Override
+        public Builder set(Builder builder, Enum<?> storageType) {
+          return builder;
+        }
+      };
+    }
+    ImmutableMap.Builder<String, StorageTypeProto> builder = ImmutableMap.builder();
+    for (StorageTypeProto storageTypeProto : StorageTypeProto.values()) {
+      builder.put(storageTypeProto.name(), storageTypeProto);
+    }
+    final ImmutableMap<String, StorageTypeProto> name2ProtoEnum = builder.build();
+    return new StorageTypeSetter() {
+
+      @Override
+      public Builder set(Builder builder, Enum<?> storageType) {
+        Object protoEnum = name2ProtoEnum.get(storageType.name());
+        try {
+          setStorageTypeMethod.invoke(builder, protoEnum);
+        } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+          throw new RuntimeException(e);
+        }
+        return builder;
+      }
+    };
+  }
+
+  private static FileCreater createFileCreater() {
+    for (Method method : ClientProtocol.class.getMethods()) {
+      if (method.getName().equals("create")) {
+        final Method createMethod = method;
+        Class<?>[] paramTypes = createMethod.getParameterTypes();
+        if (paramTypes[paramTypes.length - 1] == long.class) {
+          return new FileCreater() {
+
+            @Override
+            public HdfsFileStatus create(ClientProtocol namenode, String src, FsPermission masked,
+                String clientName, EnumSetWritable<CreateFlag> flag, boolean createParent,
+                short replication, long blockSize) throws IOException {
+              try {
+                return (HdfsFileStatus) createMethod.invoke(namenode, src, masked, clientName, flag,
+                  createParent, replication, blockSize);
+              } catch (IllegalAccessException e) {
+                throw new RuntimeException(e);
+              } catch (InvocationTargetException e) {
+                Throwables.propagateIfPossible(e.getTargetException(), IOException.class);
+                throw new RuntimeException(e);
+              }
+            }
+          };
+        } else {
+          try {
+            Class<?> cryptoProtocolVersionClass = Class
+                .forName("org.apache.hadoop.crypto.CryptoProtocolVersion");
+            Method supportedMethod = cryptoProtocolVersionClass.getMethod("supported");
+            final Object supported = supportedMethod.invoke(null);
+            return new FileCreater() {
+
+              @Override
+              public HdfsFileStatus create(ClientProtocol namenode, String src, FsPermission masked,
+                  String clientName, EnumSetWritable<CreateFlag> flag, boolean createParent,
+                  short replication, long blockSize) throws IOException {
+                try {
+                  return (HdfsFileStatus) createMethod.invoke(namenode, src, masked, clientName,
+                    flag, createParent, replication, blockSize, supported);
+                } catch (IllegalAccessException e) {
+                  throw new RuntimeException(e);
+                } catch (InvocationTargetException e) {
+                  Throwables.propagateIfPossible(e.getTargetException(), IOException.class);
+                  throw new RuntimeException(e);
+                }
+              }
+            };
+          } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
+              | InvocationTargetException e) {
+            throw new Error(e);
+          }
+        }
+      }
+    }
+    throw new Error("No create method found for " + ClientProtocol.class.getName());
+  }
+
+  // cancel the processing if DFSClient is already closed.
+  static final class CancelOnClose implements CancelableProgressable {
+
+    private final DFSClient client;
+
+    public CancelOnClose(DFSClient client) {
+      this.client = client;
+    }
+
+    @Override
+    public boolean progress() {
+      return DFS_CLIENT_ADAPTOR.isClientRunning(client);
+    }
+
+  }
+
+  static {
+    try {
+      CREATE_CHECKSUM = DFSClient.Conf.class.getDeclaredMethod("createChecksum");
+      CREATE_CHECKSUM.setAccessible(true);
+    } catch (NoSuchMethodException e) {
+      throw new Error(e);
+    }
+
+    PIPELINE_ACK_STATUS_GETTER = createPipelineAckStatusGetter();
+    STORAGE_TYPE_SETTER = createStorageTypeSetter();
+    FILE_CREATER = createFileCreater();
+    LEASE_MANAGER = createLeaseManager();
+    DFS_CLIENT_ADAPTOR = createDFSClientAdaptor();
+  }
+
+  static void beginFileLease(DFSClient client, String src, long inodeId) {
+    LEASE_MANAGER.begin(client, src, inodeId);
+  }
+
+  static void endFileLease(DFSClient client, String src, long inodeId) {
+    LEASE_MANAGER.end(client, src, inodeId);
+  }
+
+  static DataChecksum createChecksum(DFSClient client) {
+    try {
+      return (DataChecksum) CREATE_CHECKSUM.invoke(client.getConf());
+    } catch (IllegalAccessException | InvocationTargetException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  static Status getStatus(PipelineAckProto ack) {
+    return PIPELINE_ACK_STATUS_GETTER.get(ack);
+  }
+
+  private static void processWriteBlockResponse(Channel channel, final DatanodeInfo dnInfo,
+      final Promise<Channel> promise, final int timeoutMs) {
+    channel.pipeline().addLast(new IdleStateHandler(timeoutMs, 0, 0, TimeUnit.MILLISECONDS),
+      new ProtobufVarint32FrameDecoder(),
+      new ProtobufDecoder(BlockOpResponseProto.getDefaultInstance()),
+      new SimpleChannelInboundHandler<BlockOpResponseProto>() {
+
+        @Override
+        protected void channelRead0(ChannelHandlerContext ctx, BlockOpResponseProto resp)
+            throws Exception {
+          Status pipelineStatus = resp.getStatus();
+          if (PipelineAck.isRestartOOBStatus(pipelineStatus)) {
+            throw new IOException("datanode " + dnInfo + " is restarting");
+          }
+          String logInfo = "ack with firstBadLink as " + resp.getFirstBadLink();
+          if (resp.getStatus() != Status.SUCCESS) {
+            if (resp.getStatus() == Status.ERROR_ACCESS_TOKEN) {
+              throw new InvalidBlockTokenException("Got access token error" + ", status message "
+                  + resp.getMessage() + ", " + logInfo);
+            } else {
+              throw new IOException("Got error" + ", status=" + resp.getStatus().name()
+                  + ", status message " + resp.getMessage() + ", " + logInfo);
+            }
+          }
+          // success
+          ChannelPipeline p = ctx.pipeline();
+          while (p.first() != null) {
+            p.removeFirst();
+          }
+          // Disable auto read here. Enable it after we setup the streaming pipeline in
+          // FanOutOneBLockAsyncDFSOutput.
+          ctx.channel().config().setAutoRead(false);
+          promise.trySuccess(ctx.channel());
+        }
+
+        @Override
+        public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+          promise.tryFailure(new IOException("connection to " + dnInfo + " is closed"));
+        }
+
+        @Override
+        public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
+          if (evt instanceof IdleStateEvent
+              && ((IdleStateEvent) evt).state() == IdleState.READER_IDLE) {
+            promise
+                .tryFailure(new IOException("Timeout(" + timeoutMs + "ms) waiting for response"));
+          } else {
+            super.userEventTriggered(ctx, evt);
+          }
+        }
+
+        @Override
+        public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+          promise.tryFailure(cause);
+        }
+      });
+  }
+
+  private static void requestWriteBlock(Channel channel, Enum<?> storageType,
+      OpWriteBlockProto.Builder writeBlockProtoBuilder) throws IOException {
+    // TODO: SASL negotiation. should be done using a netty Handler.
+    OpWriteBlockProto proto = STORAGE_TYPE_SETTER.set(writeBlockProtoBuilder, storageType).build();
+    int protoLen = proto.getSerializedSize();
+    ByteBuf buffer = channel.alloc()
+        .buffer(3 + CodedOutputStream.computeRawVarint32Size(protoLen) + protoLen);
+    buffer.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+    buffer.writeByte(Op.WRITE_BLOCK.code);
+    proto.writeDelimitedTo(new ByteBufOutputStream(buffer));
+    channel.writeAndFlush(buffer);
+  }
+
+  private static List<Future<Channel>> connectToDataNodes(Configuration conf, String clientName,
+      LocatedBlock locatedBlock, long maxBytesRcvd, long latestGS, BlockConstructionStage stage,
+      DataChecksum summer, EventLoop eventLoop) {
+    Enum<?>[] storageTypes = locatedBlock.getStorageTypes();
+    DatanodeInfo[] datanodeInfos = locatedBlock.getLocations();
+    boolean connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
+      DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
+    final int timeoutMs = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
+      HdfsServerConstants.READ_TIMEOUT);
+    ExtendedBlock blockCopy = new ExtendedBlock(locatedBlock.getBlock());
+    blockCopy.setNumBytes(locatedBlock.getBlockSize());
+    ClientOperationHeaderProto header = ClientOperationHeaderProto.newBuilder()
+        .setBaseHeader(BaseHeaderProto.newBuilder().setBlock(PBHelper.convert(blockCopy))
+            .setToken(PBHelper.convert(locatedBlock.getBlockToken())))
+        .setClientName(clientName).build();
+    ChecksumProto checksumProto = DataTransferProtoUtil.toProto(summer);
+    final OpWriteBlockProto.Builder writeBlockProtoBuilder = OpWriteBlockProto.newBuilder()
+        .setHeader(header).setStage(OpWriteBlockProto.BlockConstructionStage.valueOf(stage.name()))
+        .setPipelineSize(1).setMinBytesRcvd(locatedBlock.getBlock().getNumBytes())
+        .setMaxBytesRcvd(maxBytesRcvd).setLatestGenerationStamp(latestGS)
+        .setRequestedChecksum(checksumProto)
+        .setCachingStrategy(CachingStrategyProto.newBuilder().setDropBehind(true).build());
+    List<Future<Channel>> futureList = new ArrayList<>(datanodeInfos.length);
+    for (int i = 0; i < datanodeInfos.length; i++) {
+      final DatanodeInfo dnInfo = datanodeInfos[i];
+      // Use Enum here because StoregType is moved to another package in hadoop 2.6. Use StorageType
+      // will cause compilation error for hadoop 2.5 or before.
+      final Enum<?> storageType = storageTypes[i];
+      final Promise<Channel> promise = eventLoop.newPromise();
+      futureList.add(promise);
+      String dnAddr = dnInfo.getXferAddr(connectToDnViaHostname);
+      new Bootstrap().group(eventLoop).channel(NioSocketChannel.class)
+          .option(CONNECT_TIMEOUT_MILLIS, timeoutMs).handler(new ChannelInitializer<Channel>() {
+
+            @Override
+            protected void initChannel(Channel ch) throws Exception {
+              processWriteBlockResponse(ch, dnInfo, promise, timeoutMs);
+            }
+          }).connect(NetUtils.createSocketAddr(dnAddr)).addListener(new ChannelFutureListener() {
+
+            @Override
+            public void operationComplete(ChannelFuture future) throws Exception {
+              if (future.isSuccess()) {
+                requestWriteBlock(future.channel(), storageType, writeBlockProtoBuilder);
+              } else {
+                promise.tryFailure(future.cause());
+              }
+            }
+          });
+    }
+    return futureList;
+  }
+
+  private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem dfs, String src,
+      boolean overwrite, boolean createParent, short replication, long blockSize,
+      EventLoop eventLoop) throws IOException {
+    Configuration conf = dfs.getConf();
+    FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
+    DFSClient client = dfs.getClient();
+    String clientName = client.getClientName();
+    ClientProtocol namenode = client.getNamenode();
+    HdfsFileStatus stat = FILE_CREATER.create(namenode, src,
+      FsPermission.getFileDefault().applyUMask(FsPermission.getUMask(conf)), clientName,
+      new EnumSetWritable<CreateFlag>(
+          overwrite ? EnumSet.of(CREATE, OVERWRITE) : EnumSet.of(CREATE)),
+      createParent, replication, blockSize);
+    beginFileLease(client, src, stat.getFileId());
+    boolean succ = false;
+    LocatedBlock locatedBlock = null;
+    List<Channel> datanodeList = new ArrayList<>();
+    try {
+      DataChecksum summer = createChecksum(client);
+      locatedBlock = namenode.addBlock(src, client.getClientName(), null, null, stat.getFileId(),
+        null);
+      for (Future<Channel> future : connectToDataNodes(conf, clientName, locatedBlock, 0L, 0L,
+        PIPELINE_SETUP_CREATE, summer, eventLoop)) {
+        // fail the creation if there are connection failures since we are fail-fast. The upper
+        // layer should retry itself if needed.
+        datanodeList.add(future.syncUninterruptibly().getNow());
+      }
+      succ = true;
+      return new FanOutOneBlockAsyncDFSOutput(conf, fsUtils, dfs, client, namenode, clientName, src,
+          stat.getFileId(), locatedBlock, eventLoop, datanodeList, summer, ALLOC);
+    } finally {
+      if (!succ) {
+        for (Channel c : datanodeList) {
+          c.close();
+        }
+        endFileLease(client, src, stat.getFileId());
+        fsUtils.recoverFileLease(dfs, new Path(src), conf, new CancelOnClose(client));
+      }
+    }
+  }
+
+  /**
+   * Create a {@link FanOutOneBlockAsyncDFSOutput}. The method maybe blocked so do not call it
+   * inside {@link EventLoop}.
+   * @param eventLoop all connections to datanode will use the same event loop.
+   */
+  public static FanOutOneBlockAsyncDFSOutput createOutput(final DistributedFileSystem dfs, Path f,
+      final boolean overwrite, final boolean createParent, final short replication,
+      final long blockSize, final EventLoop eventLoop) throws IOException {
+    return new FileSystemLinkResolver<FanOutOneBlockAsyncDFSOutput>() {
+
+      @Override
+      public FanOutOneBlockAsyncDFSOutput doCall(Path p)
+          throws IOException, UnresolvedLinkException {
+        return createOutput(dfs, p.toUri().getPath(), overwrite, createParent, replication,
+          blockSize, eventLoop);
+      }
+
+      @Override
+      public FanOutOneBlockAsyncDFSOutput next(FileSystem fs, Path p) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+    }.resolve(dfs, f);
+  }
+
+  static void completeFile(DFSClient client, ClientProtocol namenode, String src, String clientName,
+      ExtendedBlock block, long fileId) {
+    for (int retry = 0;; retry++) {
+      try {
+        if (namenode.complete(src, clientName, block, fileId)) {
+          endFileLease(client, src, fileId);
+          return;
+        } else {
+          LOG.warn("complete file " + src + " not finished, retry = " + retry);
+        }
+      } catch (LeaseExpiredException e) {
+        LOG.warn("lease for file " + src + " is expired, give up", e);
+        return;
+      } catch (Exception e) {
+        LOG.warn("complete file " + src + " failed, retry = " + retry, e);
+      }
+      sleepIgnoreInterrupt(retry);
+    }
+  }
+
+  static void sleepIgnoreInterrupt(int retry) {
+    try {
+      Thread.sleep(ConnectionUtils.getPauseTime(100, retry));
+    } catch (InterruptedException e) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e9d355b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputFlushHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputFlushHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputFlushHandler.java
new file mode 100644
index 0000000..cbd0761
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/FanOutOneBlockAsyncDFSOutputFlushHandler.java
@@ -0,0 +1,61 @@
+/**
+ * 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.util;
+
+import java.nio.channels.CompletionHandler;
+import java.util.concurrent.ExecutionException;
+
+public final class FanOutOneBlockAsyncDFSOutputFlushHandler
+    implements CompletionHandler<Long, Void> {
+
+  private long size;
+
+  private Throwable error;
+
+  private boolean finished;
+
+  @Override
+  public synchronized void completed(Long result, Void attachment) {
+    size = result.longValue();
+    finished = true;
+    notifyAll();
+  }
+
+  @Override
+  public synchronized void failed(Throwable exc, Void attachment) {
+    error = exc;
+    finished = true;
+    notifyAll();
+  }
+
+  public synchronized long get() throws InterruptedException, ExecutionException {
+    while (!finished) {
+      wait();
+    }
+    if (error != null) {
+      throw new ExecutionException(error);
+    }
+    return size;
+  }
+
+  public void reset() {
+    size = 0L;
+    error = null;
+    finished = false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e9d355b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java
new file mode 100644
index 0000000..0e9f42e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFanOutOneBlockAsyncDFSOutput.java
@@ -0,0 +1,190 @@
+/**
+ * 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.util;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import io.netty.channel.EventLoop;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category({ MiscTests.class, MediumTests.class })
+public class TestFanOutOneBlockAsyncDFSOutput {
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static DistributedFileSystem FS;
+
+  private static EventLoopGroup EVENT_LOOP_GROUP;
+
+  private static int READ_TIMEOUT_MS = 2000;
+
+  @Rule
+  public TestName name = new TestName();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Logger.getLogger("org.apache.hadoop.hdfs.StateChange").setLevel(Level.DEBUG);
+    Logger.getLogger("BlockStateChange").setLevel(Level.DEBUG);
+    TEST_UTIL.getConfiguration().setInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, READ_TIMEOUT_MS);
+    TEST_UTIL.startMiniDFSCluster(3);
+    FS = TEST_UTIL.getDFSCluster().getFileSystem();
+    EVENT_LOOP_GROUP = new NioEventLoopGroup();
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException, InterruptedException {
+    if (EVENT_LOOP_GROUP != null) {
+      EVENT_LOOP_GROUP.shutdownGracefully().sync();
+    }
+    TEST_UTIL.shutdownMiniDFSCluster();
+  }
+
+  private void writeAndVerify(EventLoop eventLoop, Path f, final FanOutOneBlockAsyncDFSOutput out)
+      throws IOException, InterruptedException, ExecutionException {
+    final byte[] b = new byte[10];
+    ThreadLocalRandom.current().nextBytes(b);
+    final FanOutOneBlockAsyncDFSOutputFlushHandler handler =
+        new FanOutOneBlockAsyncDFSOutputFlushHandler();
+    eventLoop.execute(new Runnable() {
+
+      @Override
+      public void run() {
+        out.write(b, 0, b.length);
+        out.flush(null, handler, false);
+      }
+    });
+    assertEquals(b.length, handler.get());
+    out.close();
+    assertEquals(b.length, FS.getFileStatus(f).getLen());
+    byte[] actual = new byte[b.length];
+    try (FSDataInputStream in = FS.open(f)) {
+      in.readFully(actual);
+    }
+    assertArrayEquals(b, actual);
+  }
+
+  @Test
+  public void test() throws IOException, InterruptedException, ExecutionException {
+    Path f = new Path("/" + name.getMethodName());
+    EventLoop eventLoop = EVENT_LOOP_GROUP.next();
+    final FanOutOneBlockAsyncDFSOutput out =
+        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
+          FS.getDefaultBlockSize(), eventLoop);
+    writeAndVerify(eventLoop, f, out);
+  }
+
+  @Test
+  public void testRecover() throws IOException, InterruptedException, ExecutionException {
+    Path f = new Path("/" + name.getMethodName());
+    EventLoop eventLoop = EVENT_LOOP_GROUP.next();
+    final FanOutOneBlockAsyncDFSOutput out =
+        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
+          FS.getDefaultBlockSize(), eventLoop);
+    final byte[] b = new byte[10];
+    ThreadLocalRandom.current().nextBytes(b);
+    final FanOutOneBlockAsyncDFSOutputFlushHandler handler =
+        new FanOutOneBlockAsyncDFSOutputFlushHandler();
+    eventLoop.execute(new Runnable() {
+
+      @Override
+      public void run() {
+        out.write(b, 0, b.length);
+        out.flush(null, handler, false);
+      }
+    });
+    handler.get();
+    // restart one datanode which causes one connection broken
+    TEST_UTIL.getDFSCluster().restartDataNode(0);
+    handler.reset();
+    eventLoop.execute(new Runnable() {
+
+      @Override
+      public void run() {
+        out.write(b, 0, b.length);
+        out.flush(null, handler, false);
+      }
+    });
+    try {
+      handler.get();
+      fail("flush should fail");
+    } catch (ExecutionException e) {
+      // we restarted one datanode so the flush should fail
+      e.printStackTrace();
+    }
+    out.recoverAndClose(null);
+    assertEquals(b.length, FS.getFileStatus(f).getLen());
+    byte[] actual = new byte[b.length];
+    try (FSDataInputStream in = FS.open(f)) {
+      in.readFully(actual);
+    }
+    assertArrayEquals(b, actual);
+  }
+
+  @Test
+  public void testHeartbeat() throws IOException, InterruptedException, ExecutionException {
+    Path f = new Path("/" + name.getMethodName());
+    EventLoop eventLoop = EVENT_LOOP_GROUP.next();
+    final FanOutOneBlockAsyncDFSOutput out =
+        FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
+          FS.getDefaultBlockSize(), eventLoop);
+    Thread.sleep(READ_TIMEOUT_MS * 2);
+    // the connection to datanode should still alive.
+    writeAndVerify(eventLoop, f, out);
+  }
+
+  /**
+   * This is important for fencing when recover from RS crash.
+   */
+  @Test
+  public void testCreateParentFailed() throws IOException {
+    Path f = new Path("/" + name.getMethodName() + "/test");
+    EventLoop eventLoop = EVENT_LOOP_GROUP.next();
+    try {
+      FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3,
+        FS.getDefaultBlockSize(), eventLoop);
+      fail("should fail with parent does not exist");
+    } catch (RemoteException e) {
+      assertTrue(e.unwrapRemoteException() instanceof FileNotFoundException);
+    }
+  }
+}


[36/37] hbase git commit: HBASE-15349 Update surefire version to 2.19.1. (Apekshit)

Posted by sy...@apache.org.
HBASE-15349 Update surefire version to 2.19.1. (Apekshit)

Signed-off-by: stack <st...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 3c660e2a0f436a52a9bbdfb7c6dd82bf67097639
Parents: c528894
Author: Apekshit <ap...@gmail.com>
Authored: Fri Feb 26 12:01:08 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 26 12:26:40 2016 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3c660e2a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 82eff70..b3fa787 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1239,7 +1239,7 @@
     <procedure.test.jar>hbase-procedure-${project.version}-tests.jar</procedure.test.jar>
     <it.test.jar>hbase-it-${project.version}-tests.jar</it.test.jar>
     <annotations.test.jar>hbase-annotations-${project.version}-tests.jar</annotations.test.jar>
-    <surefire.version>2.18.1</surefire.version>
+    <surefire.version>2.19.1</surefire.version>
     <surefire.provider>surefire-junit47</surefire.provider>
     <!-- default: run small & medium, medium with 2 threads -->
     <surefire.skipFirstPart>false</surefire.skipFirstPart>


[37/37] hbase git commit: HBASE-15351 Fix description of hbase.bucketcache.size in hbase-default.xml

Posted by sy...@apache.org.
HBASE-15351 Fix description of hbase.bucketcache.size in hbase-default.xml


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

Branch: refs/heads/hbase-12439
Commit: 8f6e29785abff6b3205d7058d9b88c1cf27faa29
Parents: 3c660e2
Author: stack <st...@apache.org>
Authored: Fri Feb 26 13:49:26 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 26 13:49:26 2016 -0800

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        | 28 ++++++++++++--------
 1 file changed, 17 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8f6e2978/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index e446a24..e50e89e 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -857,29 +857,35 @@ possible configurations would overwhelm and obscure the important.
     <name>hbase.bucketcache.ioengine</name>
     <value></value>
     <description>Where to store the contents of the bucketcache. One of: heap,
-      offheap, or file. If a file, set it to file:PATH_TO_FILE. See
-      http://hbase.apache.org/book.html#offheap.blockcache for more information.
+    offheap, or file. If a file, set it to file:PATH_TO_FILE. See
+    http://hbase.apache.org/book.html#offheap.blockcache for more information.
     </description>
   </property>
   <property>
     <name>hbase.bucketcache.combinedcache.enabled</name>
     <value>true</value>
     <description>Whether or not the bucketcache is used in league with the LRU
-      on-heap block cache. In this mode, indices and blooms are kept in the LRU
-      blockcache and the data blocks are kept in the bucketcache.</description>
+    on-heap block cache. In this mode, indices and blooms are kept in the LRU
+    blockcache and the data blocks are kept in the bucketcache.</description>
   </property>
   <property>
     <name>hbase.bucketcache.size</name>
-    <value>65536</value>
-    <description>The size of the buckets for the bucketcache if you only use a single size.
-      Defaults to the default blocksize, which is 64 * 1024.</description>
+    <value></value>
+    <description>A float that EITHER represents a percentage of total heap memory
+    size to give to the cache (if &lt; 1.0) OR, it is the total capacity in
+    megabytes of BucketCache. Default: 0.0</description>
   </property>
   <property>
     <name>hbase.bucketcache.sizes</name>
     <value></value>
-    <description>A comma-separated list of sizes for buckets for the bucketcache
-      if you use multiple sizes. Should be a list of block sizes in order from smallest
-      to largest. The sizes you use will depend on your data access patterns.</description>
+    <description>A comma-separated list of sizes for buckets for the bucketcache.
+    Can be multiple sizes. List block sizes in order from smallest to largest.
+    The sizes you use will depend on your data access patterns.
+    Must be a multiple of 1024 else you will run into
+    'java.io.IOException: Invalid HFile block magic' when you go to read from cache.
+    If you specify no values here, then you pick up the default bucketsizes set
+    in code (See BucketAllocator#DEFAULT_BUCKET_SIZES). 
+  </description>
   </property>
   <property>
       <name>hfile.format.version</name>
@@ -907,7 +913,7 @@ possible configurations would overwhelm and obscure the important.
       <name>hbase.rs.cacheblocksonwrite</name>
       <value>false</value>
       <description>Whether an HFile block should be added to the block cache when the
-          block is finished.</description>
+        block is finished.</description>
   </property>
   <property>
     <name>hbase.rpc.timeout</name>


[22/37] hbase git commit: HBASE-15311 Prevent NPE in BlockCacheViewTmpl.

Posted by sy...@apache.org.
HBASE-15311 Prevent NPE in BlockCacheViewTmpl.

Signed-off-by: stack <st...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 75c57a04ddad2d7cf3435df1eba13541775319fb
Parents: 40c5591
Author: Samir Ahmic <sa...@personal.com>
Authored: Tue Feb 23 11:34:09 2016 +0100
Committer: stack <st...@apache.org>
Committed: Thu Feb 25 15:23:28 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/75c57a04/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
index fa55f6a..c6d7a61 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
@@ -44,7 +44,7 @@ org.apache.hadoop.util.StringUtils;
   if (bcn.equals("L1")) {
     bc = bcs == null || bcs.length == 0? bc: bcs[0];
   } else {
-    if (bcs.length < 2) {
+    if (bcs == null || bcs.length < 2) {
       System.out.println("There is no L2 block cache");
       return;
     }


[03/37] hbase git commit: HBASE-15232 Handle region location cache mgmt in AsyncProcess for multi()'s

Posted by sy...@apache.org.
HBASE-15232 Handle region location cache mgmt in AsyncProcess for multi()'s

Further investigation after HBASE-15221 lead to some findings that
AsyncProcess should have been managing the contents of the region
location cache, appropriately clearing it when necessary (e.g. an
RPC to a server fails because the server doesn't host that region)

For multi() RPCs, the tableName argument is null since there is no
single table that the updates are destined to. This inadvertently
caused the existing region location cache updates to fail on 1.x
branches. AsyncProcess needs to handle when tableName is null
and perform the necessary cache evictions.

As such, much of the new retry logic in HTableMultiplexer is
unnecessary and is removed with this commit. Getters which were
added as a part of testing were left since that are mostly
harmless and should contain no negative impact.

Signed-off-by: stack <st...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: a8073c4a9819953a2dd13a26bb4dd9405ac8750c
Parents: 5e50112
Author: Josh Elser <el...@apache.org>
Authored: Mon Feb 8 14:25:37 2016 -0500
Committer: stack <st...@apache.org>
Committed: Mon Feb 22 22:03:14 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncProcess.java       |  11 +-
 .../hadoop/hbase/client/HTableMultiplexer.java  |  23 ++--
 .../hbase/exceptions/ClientExceptionsUtil.java  |   6 +-
 .../client/TestHTableMultiplexerViaMocks.java   | 117 -------------------
 .../client/TestHTableMultiplexerFlushCache.java |  60 ++++++++++
 5 files changed, 81 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a8073c4a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index 4ceb89a..65c15ce 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -1360,8 +1360,15 @@ class AsyncProcess {
           errorsByServer.reportServerError(server);
           canRetry = errorsByServer.canTryMore(numAttempt);
         }
-        connection.updateCachedLocations(
-            tableName, region, actions.get(0).getAction().getRow(), throwable, server);
+        if (null == tableName && ClientExceptionsUtil.isMetaClearingException(throwable)) {
+          // For multi-actions, we don't have a table name, but we want to make sure to clear the
+          // cache in case there were location-related exceptions. We don't to clear the cache
+          // for every possible exception that comes through, however.
+          connection.clearCaches(server);
+        } else {
+          connection.updateCachedLocations(
+              tableName, region, actions.get(0).getAction().getRow(), throwable, server);
+        }
         failureCount += actions.size();
 
         for (Action<Row> action : actions) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8073c4a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index 13e9b85..f1bbcb3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -194,19 +194,6 @@ public class HTableMultiplexer {
    * @return true if the request can be accepted by its corresponding buffer queue.
    */
   public boolean put(final TableName tableName, final Put put, int maxAttempts) {
-    return _put(tableName, put, maxAttempts, false);
-  }
-
-  /**
-   * Internal "put" which exposes a boolean flag to control whether or not the region location
-   * cache should be reloaded when trying to queue the {@link Put}.
-   * @param tableName Destination table for the Put
-   * @param put The Put to send
-   * @param maxAttempts Number of attempts to retry the {@code put}
-   * @param reloadCache Should the region location cache be reloaded
-   * @return true if the request was accepted in the queue, otherwise false
-   */
-  boolean _put(final TableName tableName, final Put put, int maxAttempts, boolean reloadCache) {
     if (maxAttempts <= 0) {
       return false;
     }
@@ -215,7 +202,9 @@ public class HTableMultiplexer {
       HTable.validatePut(put, maxKeyValueSize);
       // Allow mocking to get at the connection, but don't expose the connection to users.
       ClusterConnection conn = (ClusterConnection) getConnection();
-      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), reloadCache);
+      // AsyncProcess in the FlushWorker should take care of refreshing the location cache
+      // as necessary. We shouldn't have to do that here.
+      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false);
       if (loc != null) {
         // Add the put pair into its corresponding queue.
         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
@@ -512,12 +501,16 @@ public class HTableMultiplexer {
         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
       }
 
+      // HBASE-12198, HBASE-15221, HBASE-15232: AsyncProcess should be responsible for updating
+      // the region location cache when the Put original failed with some exception. If we keep
+      // re-trying the same Put to the same location, AsyncProcess isn't doing the right stuff
+      // that we expect it to.
       getExecutor().schedule(new Runnable() {
         @Override
         public void run() {
           boolean succ = false;
           try {
-            succ = FlushWorker.this.getMultiplexer()._put(tableName, failedPut, retryCount, true);
+            succ = FlushWorker.this.getMultiplexer().put(tableName, failedPut, retryCount);
           } finally {
             FlushWorker.this.getRetryInQueue().decrementAndGet();
             if (!succ) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8073c4a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
index ebf1499..1d6f5d6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.exceptions;
 
 import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.MultiActionResultTooLarge;
+import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionTooBusyException;
 import org.apache.hadoop.hbase.RetryImmediatelyException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -40,14 +41,15 @@ public final class ClientExceptionsUtil {
     if (cur == null) {
       return true;
     }
-    return !isSpecialException(cur) || (cur instanceof RegionMovedException);
+    return !isSpecialException(cur) || (cur instanceof RegionMovedException)
+        || cur instanceof NotServingRegionException;
   }
 
   public static boolean isSpecialException(Throwable cur) {
     return (cur instanceof RegionMovedException || cur instanceof RegionOpeningException
         || cur instanceof RegionTooBusyException || cur instanceof ThrottlingException
         || cur instanceof MultiActionResultTooLarge || cur instanceof RetryImmediatelyException
-        || cur instanceof CallQueueTooBigException);
+        || cur instanceof CallQueueTooBigException || cur instanceof NotServingRegionException);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8073c4a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
index 38ddeb9..7e68c21 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
@@ -16,34 +16,17 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.*;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HTableMultiplexer.FlushWorker;
-import org.apache.hadoop.hbase.client.HTableMultiplexer.PutStatus;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 import java.io.IOException;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -52,112 +35,19 @@ import static org.mockito.Mockito.when;
 @Category(SmallTests.class)
 public class TestHTableMultiplexerViaMocks {
 
-  private static final int NUM_RETRIES = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
   private HTableMultiplexer mockMultiplexer;
   private ClusterConnection mockConnection;
-  private HRegionLocation mockRegionLocation;
-  private HRegionInfo mockRegionInfo;
-
-  private TableName tableName;
-  private Put put;
 
   @Before
   public void setupTest() {
     mockMultiplexer = mock(HTableMultiplexer.class);
     mockConnection = mock(ClusterConnection.class);
-    mockRegionLocation = mock(HRegionLocation.class);
-    mockRegionInfo = mock(HRegionInfo.class);
-
-    tableName = TableName.valueOf("my_table");
-    put = new Put(getBytes("row1"));
-    put.addColumn(getBytes("f1"), getBytes("q1"), getBytes("v11"));
-    put.addColumn(getBytes("f1"), getBytes("q2"), getBytes("v12"));
-    put.addColumn(getBytes("f2"), getBytes("q1"), getBytes("v21"));
 
     // Call the real put(TableName, Put, int) method
     when(mockMultiplexer.put(any(TableName.class), any(Put.class), anyInt())).thenCallRealMethod();
 
     // Return the mocked ClusterConnection
     when(mockMultiplexer.getConnection()).thenReturn(mockConnection);
-
-    // Return the regionInfo from the region location
-    when(mockRegionLocation.getRegionInfo()).thenReturn(mockRegionInfo);
-
-    // Make sure this RegionInfo points to our table
-    when(mockRegionInfo.getTable()).thenReturn(tableName);
-  }
-
-  @Test public void useCacheOnInitialPut() throws Exception {
-    mockMultiplexer.put(tableName, put, NUM_RETRIES);
-
-    verify(mockMultiplexer)._put(tableName, put, NUM_RETRIES, false);
-  }
-
-  @Test public void nonNullLocationQueuesPut() throws Exception {
-    final LinkedBlockingQueue<PutStatus> queue = new LinkedBlockingQueue<>();
-
-    // Call the real method for _put(TableName, Put, int, boolean)
-    when(mockMultiplexer._put(any(TableName.class), any(Put.class), anyInt(), anyBoolean())).thenCallRealMethod();
-
-    // Return a region location
-    when(mockConnection.getRegionLocation(tableName, put.getRow(), false)).thenReturn(mockRegionLocation);
-    when(mockMultiplexer.getQueue(mockRegionLocation)).thenReturn(queue);
-
-    assertTrue("Put should have been queued", mockMultiplexer.put(tableName, put, NUM_RETRIES));
-
-    assertEquals(1, queue.size());
-    final PutStatus ps = queue.take();
-    assertEquals(put, ps.put);
-    assertEquals(mockRegionInfo, ps.regionInfo);
-  }
-
-  @Test public void ignoreCacheOnRetriedPut() throws Exception {
-    FlushWorker mockFlushWorker = mock(FlushWorker.class);
-    ScheduledExecutorService mockExecutor = mock(ScheduledExecutorService.class);
-    final AtomicInteger retryInQueue = new AtomicInteger(0);
-    final AtomicLong totalFailedPuts = new AtomicLong(0L);
-    final int maxRetryInQueue = 20;
-    final long delay = 100L;
-
-    final PutStatus ps = new PutStatus(mockRegionInfo, put, NUM_RETRIES);
-
-    // Call the real resubmitFailedPut(PutStatus, HRegionLocation) method
-    when(mockFlushWorker.resubmitFailedPut(any(PutStatus.class), any(HRegionLocation.class))).thenCallRealMethod();
-    // Succeed on the re-submit without caching
-    when(mockMultiplexer._put(tableName, put, NUM_RETRIES - 1, true)).thenReturn(true);
-
-    // Stub out the getters for resubmitFailedPut(PutStatus, HRegionLocation)
-    when(mockFlushWorker.getExecutor()).thenReturn(mockExecutor);
-    when(mockFlushWorker.getNextDelay(anyInt())).thenReturn(delay);
-    when(mockFlushWorker.getMultiplexer()).thenReturn(mockMultiplexer);
-    when(mockFlushWorker.getRetryInQueue()).thenReturn(retryInQueue);
-    when(mockFlushWorker.getMaxRetryInQueue()).thenReturn(maxRetryInQueue);
-    when(mockFlushWorker.getTotalFailedPutCount()).thenReturn(totalFailedPuts);
-
-    // When a Runnable is scheduled, run that Runnable
-    when(mockExecutor.schedule(any(Runnable.class), eq(delay), eq(TimeUnit.MILLISECONDS))).thenAnswer(
-        new Answer<Void>() {
-          @Override
-          public Void answer(InvocationOnMock invocation) throws Throwable {
-            // Before we run this, should have one retry in progress.
-            assertEquals(1L, retryInQueue.get());
-
-            Object[] args = invocation.getArguments();
-            assertEquals(3, args.length);
-            assertTrue("Argument should be an instance of Runnable", args[0] instanceof Runnable);
-            Runnable runnable = (Runnable) args[0];
-            runnable.run();
-            return null;
-          }
-        });
-
-    // The put should be rescheduled
-    assertTrue("Put should have been rescheduled", mockFlushWorker.resubmitFailedPut(ps, mockRegionLocation));
-
-    verify(mockMultiplexer)._put(tableName, put, NUM_RETRIES - 1, true);
-    assertEquals(0L, totalFailedPuts.get());
-    // Net result should be zero (added one before rerunning, subtracted one after running).
-    assertEquals(0L, retryInQueue.get());
   }
 
   @SuppressWarnings("deprecation")
@@ -183,11 +73,4 @@ public class TestHTableMultiplexerViaMocks {
     // We should not close it again
     verify(mockConnection, times(0)).close();
   }
-
-  /**
-   * @return UTF-8 byte representation for {@code str}
-   */
-  private static byte[] getBytes(String str) {
-    return str.getBytes(UTF_8);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a8073c4a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
index 063e376..05c9caa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
@@ -24,16 +24,21 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 @Category({ LargeTests.class, ClientTests.class })
@@ -115,4 +120,59 @@ public class TestHTableMultiplexerFlushCache {
       checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2);
     }
   }
+
+  @Test
+  public void testOnRegionMove() throws Exception {
+    // This test is doing near exactly the same thing that testOnRegionChange but avoiding the
+    // potential to get a ConnectionClosingException. By moving the region, we can be certain that
+    // the connection is still valid and that the implementation is correctly handling an invalid
+    // Region cache (and not just tearing down the entire connection).
+    TableName TABLE = TableName.valueOf("testOnRegionMove");
+    final int NUM_REGIONS = 10;
+    HTable htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3,
+      Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS);
+
+    HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(),
+      PER_REGIONSERVER_QUEUE_SIZE);
+
+    final RegionLocator regionLocator = TEST_UTIL.getConnection().getRegionLocator(TABLE);
+    Pair<byte[][],byte[][]> startEndRows = regionLocator.getStartEndKeys();
+    byte[] row = startEndRows.getFirst()[1];
+    assertTrue("2nd region should not start with empty row", row != null && row.length > 0);
+
+    Put put = new Put(row).addColumn(FAMILY, QUALIFIER1, VALUE1);
+    assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
+
+    checkExistence(htable, row, FAMILY, QUALIFIER1, VALUE1);
+
+    final HRegionLocation loc = regionLocator.getRegionLocation(row);
+    final MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
+    // The current server for the region we're writing to
+    final ServerName originalServer = loc.getServerName();
+    ServerName newServer = null;
+    // Find a new server to move that region to
+    for (int i = 0; i < SLAVES; i++) {
+      HRegionServer rs = hbaseCluster.getRegionServer(0);
+      if (!rs.getServerName().equals(originalServer.getServerName())) {
+        newServer = rs.getServerName();
+        break;
+      }
+    }
+    assertNotNull("Did not find a new RegionServer to use", newServer);
+
+    // Move the region
+    LOG.info("Moving " + loc.getRegionInfo().getEncodedName() + " from " + originalServer
+        +  " to " + newServer);
+    TEST_UTIL.getHBaseAdmin().move(loc.getRegionInfo().getEncodedNameAsBytes(),
+        Bytes.toBytes(newServer.getServerName()));
+
+    TEST_UTIL.waitUntilAllRegionsAssigned(TABLE);
+
+    // Send a new Put
+    put = new Put(row).addColumn(FAMILY, QUALIFIER2, VALUE2);
+    assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
+
+    // We should see the update make it to the new server eventually
+    checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2);
+  }
 }


[17/37] hbase git commit: HBASE-15319 clearJmxCache does not take effect actually

Posted by sy...@apache.org.
HBASE-15319 clearJmxCache does not take effect actually


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

Branch: refs/heads/hbase-12439
Commit: a3b4575f700c20167a6ab0b774d8a2c9cd3916af
Parents: 630a658
Author: Elliott Clark <ec...@apache.org>
Authored: Wed Feb 24 09:02:06 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Feb 24 16:29:05 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3b4575f/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
index 95734ba..8fcf623 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
@@ -52,7 +52,7 @@ public class JmxCacheBuster {
   public static void clearJmxCache() {
     //If there are more then 100 ms before the executor will run then everything should be merged.
     ScheduledFuture future = fut.get();
-    if ((future == null || (!future.isDone() && future.getDelay(TimeUnit.MILLISECONDS) > 100))) {
+    if ((future != null && (!future.isDone() && future.getDelay(TimeUnit.MILLISECONDS) > 100))) {
       // BAIL OUT
       return;
     }


[30/37] hbase git commit: Revert "HBASE-15128 Disable region splits and merges switch in master"

Posted by sy...@apache.org.
Revert "HBASE-15128 Disable region splits and merges switch in master"

This reverts commit 24d481c5803e69a6190339cd8bb218b2c4585459.


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

Branch: refs/heads/hbase-12439
Commit: bf4fcc30c62395e8db9fe52fde07c752f9e00e54
Parents: 24d481c
Author: chenheng <ch...@apache.org>
Authored: Fri Feb 26 08:52:12 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Fri Feb 26 08:52:12 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |   27 +-
 .../hbase/client/ConnectionImplementation.java  |   14 -
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   31 -
 .../hadoop/hbase/protobuf/RequestConverter.java |   49 -
 .../hbase/zookeeper/ZooKeeperWatcher.java       |   10 -
 .../hbase/protobuf/generated/MasterProtos.java  | 4304 ++++--------------
 .../protobuf/generated/SnapshotProtos.java      |  500 +-
 .../protobuf/generated/ZooKeeperProtos.java     |  462 +-
 hbase-protocol/src/main/protobuf/Master.proto   |   36 -
 .../src/main/protobuf/ZooKeeper.proto           |    7 -
 .../hadoop/hbase/master/AssignmentManager.java  |   10 -
 .../org/apache/hadoop/hbase/master/HMaster.java |   28 -
 .../hadoop/hbase/master/MasterRpcServices.java  |   42 -
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   35 -
 .../zookeeper/SplitOrMergeTrackerManager.java   |  151 -
 .../hbase/client/TestSplitOrMergeStatus.java    |  198 -
 hbase-shell/src/main/ruby/hbase/admin.rb        |   32 -
 hbase-shell/src/main/ruby/shell.rb              |    2 -
 .../ruby/shell/commands/splitormerge_enabled.rb |   41 -
 .../ruby/shell/commands/splitormerge_switch.rb  |   43 -
 20 files changed, 1200 insertions(+), 4822 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index c3b524b..d7b52d5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -1678,28 +1678,11 @@ public interface Admin extends Abortable, Closeable {
   List<SecurityCapability> getSecurityCapabilities() throws IOException;
 
   /**
-   * Turn the Split or Merge switches on or off.
-   *
-   * @param enabled enabled or not
-   * @param synchronous If true, it waits until current split() call, if outstanding, to return.
-   * @param switchTypes switchType list {@link MasterSwitchType}
-   * @return Previous switch value array
-   */
-  boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
-                                   final MasterSwitchType... switchTypes) throws IOException;
-
-  /**
-   * Query the current state of the switch
-   *
-   * @return true if the switch is enabled, false otherwise.
-   */
-  boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException;
-
-  /**
    * Currently, there are only two compact types:
    * {@code NORMAL} means do store files compaction;
    * {@code MOB} means do mob files compaction.
    * */
+
   @InterfaceAudience.Public
   @InterfaceStability.Unstable
   public enum CompactType {
@@ -1709,12 +1692,4 @@ public interface Admin extends Abortable, Closeable {
 
     CompactType(int value) {}
   }
-  
-  @InterfaceAudience.Public
-  @InterfaceStability.Evolving
-  public enum MasterSwitchType {
-    SPLIT,
-    MERGE
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 64eb9fb..dfa9937 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1742,20 +1742,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }
 
       @Override
-      public MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(
-        RpcController controller, MasterProtos.SetSplitOrMergeEnabledRequest request)
-        throws ServiceException {
-        return stub.setSplitOrMergeEnabled(controller, request);
-      }
-
-      @Override
-      public MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(
-        RpcController controller, MasterProtos.IsSplitOrMergeEnabledRequest request)
-              throws ServiceException {
-        return stub.isSplitOrMergeEnabled(controller, request);
-      }
-
-      @Override
       public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller,
           IsNormalizerEnabledRequest request) throws ServiceException {
         return stub.isNormalizerEnabled(controller, request);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index c2a0bb8..db94ff4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -89,7 +89,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescripti
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
@@ -3379,36 +3378,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  @Override
-  public boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
-                                          final MasterSwitchType... switchTypes)
-    throws IOException {
-    return executeCallable(new MasterCallable<boolean[]>(getConnection()) {
-      @Override
-      public boolean[] call(int callTimeout) throws ServiceException {
-        MasterProtos.SetSplitOrMergeEnabledResponse response = master.setSplitOrMergeEnabled(null,
-          RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchTypes));
-        boolean[] result = new boolean[switchTypes.length];
-        int i = 0;
-        for (Boolean prevValue : response.getPrevValueList()) {
-          result[i++] = prevValue;
-        }
-        return result;
-      }
-    });
-  }
-
-  @Override
-  public boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
-      @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        return master.isSplitOrMergeEnabled(null,
-          RequestConverter.buildIsSplitOrMergeEnabledRequest(switchType)).getEnabled();
-      }
-    });
-  }
-
   private HRegionInfo getMobRegionInfo(TableName tableName) {
     return new HRegionInfo(tableName, Bytes.toBytes(".mob"),
             HConstants.EMPTY_END_ROW, false, 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 99e993d..572d92c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Action;
-import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -77,7 +76,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
@@ -97,7 +95,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabled
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
@@ -106,7 +103,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequ
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
@@ -1696,49 +1692,4 @@ public final class RequestConverter {
   public static SetNormalizerRunningRequest buildSetNormalizerRunningRequest(boolean on) {
     return SetNormalizerRunningRequest.newBuilder().setOn(on).build();
   }
-
-  /**
-   * Creates a protocol buffer IsSplitOrMergeEnabledRequest
-   *
-   * @param switchType see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}
-   * @return a IsSplitOrMergeEnabledRequest
-   */
-  public static IsSplitOrMergeEnabledRequest buildIsSplitOrMergeEnabledRequest(
-    Admin.MasterSwitchType switchType) {
-    IsSplitOrMergeEnabledRequest.Builder builder = IsSplitOrMergeEnabledRequest.newBuilder();
-    builder.setSwitchType(convert(switchType));
-    return builder.build();
-  }
-
-  /**
-   * Creates a protocol buffer SetSplitOrMergeEnabledRequest
-   *
-   * @param enabled switch is enabled or not
-   * @param synchronous set switch sync?
-   * @param switchTypes see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}, it is
-   *                    a list.
-   * @return a SetSplitOrMergeEnabledRequest
-   */
-  public static SetSplitOrMergeEnabledRequest buildSetSplitOrMergeEnabledRequest(boolean enabled,
-    boolean synchronous, Admin.MasterSwitchType... switchTypes) {
-    SetSplitOrMergeEnabledRequest.Builder builder = SetSplitOrMergeEnabledRequest.newBuilder();
-    builder.setEnabled(enabled);
-    builder.setSynchronous(synchronous);
-    for (Admin.MasterSwitchType switchType : switchTypes) {
-      builder.addSwitchTypes(convert(switchType));
-    }
-    return builder.build();
-  }
-
-  private static MasterProtos.MasterSwitchType convert(Admin.MasterSwitchType switchType) {
-    switch (switchType) {
-      case SPLIT:
-        return MasterProtos.MasterSwitchType.SPLIT;
-      case MERGE:
-        return MasterProtos.MasterSwitchType.MERGE;
-      default:
-        break;
-    }
-    throw new UnsupportedOperationException("Unsupport switch type:" + switchType);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index b665353..36a9bc5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -115,8 +115,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
   public String balancerZNode;
   // znode containing the state of region normalizer
   private String regionNormalizerZNode;
-  // znode containing the state of all switches, currently there are split and merge child node.
-  private String switchZNode;
   // znode containing the lock for the tables
   public String tableLockZNode;
   // znode containing the state of recovering regions
@@ -432,7 +430,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
         conf.get("zookeeper.znode.balancer", "balancer"));
     regionNormalizerZNode = ZKUtil.joinZNode(baseZNode,
       conf.get("zookeeper.znode.regionNormalizer", "normalizer"));
-    switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch"));
     tableLockZNode = ZKUtil.joinZNode(baseZNode,
         conf.get("zookeeper.znode.tableLock", "table-lock"));
     recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
@@ -792,11 +789,4 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
   public String getRegionNormalizerZNode() {
     return regionNormalizerZNode;
   }
-
-  /**
-   *  @return ZK node for switch
-   * */
-  public String getSwitchZNode() {
-    return switchZNode;
-  }
 }


[35/37] hbase git commit: HBASE-15332 Document how to take advantage of HDFS-6133 in HBase

Posted by sy...@apache.org.
HBASE-15332 Document how to take advantage of HDFS-6133 in HBase

(cherry picked from commit e0a656ed50027a7d982f1eca7a8c0ee3cab47f92)


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

Branch: refs/heads/hbase-12439
Commit: c5288947ddc4abae2f4036544a775ff81538df2f
Parents: e88d943
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Thu Feb 25 13:51:26 2016 -0800
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Fri Feb 26 09:38:32 2016 -0800

----------------------------------------------------------------------
 .../asciidoc/_chapters/troubleshooting.adoc     | 22 ++++++++++++++++++++
 1 file changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c5288947/src/main/asciidoc/_chapters/troubleshooting.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/troubleshooting.adoc b/src/main/asciidoc/_chapters/troubleshooting.adoc
index 66e56b8..8b2011d 100644
--- a/src/main/asciidoc/_chapters/troubleshooting.adoc
+++ b/src/main/asciidoc/_chapters/troubleshooting.adoc
@@ -1347,6 +1347,28 @@ Settings for HDFS retries and timeouts are important to HBase.::
   Defaults are current as of Hadoop 2.3.
   Check the Hadoop documentation for the most current values and recommendations.
 
+The HBase Balancer and HDFS Balancer are incompatible::
+  The HDFS balancer attempts to spread HDFS blocks evenly among DataNodes. HBase relies
+  on compactions to restore locality after a region split or failure. These two types
+  of balancing do not work well together.
++
+In the past, the generally accepted advice was to turn off the HDFS load balancer and rely
+on the HBase balancer, since the HDFS balancer would degrade locality. This advice
+is still valid if your HDFS version is lower than 2.7.1.
++
+link:https://issues.apache.org/jira/browse/HDFS-6133[HDFS-6133] provides the ability
+to exclude a given directory from the HDFS load balancer, by setting the
+`dfs.datanode.block-pinning.enabled` property to `true` in your HDFS
+configuration and running the following hdfs command:
++
+----
+$ sudo -u hdfs hdfs balancer -exclude /hbase
+----
++
+NOTE: HDFS-6133 is available in HDFS 2.7.0 and higher, but HBase does not support
+running on HDFS 2.7.0, so you must be using HDFS 2.7.1 or higher to use this feature
+with HBase.
+
 .Connection Timeouts
 Connection timeouts occur between the client (HBASE) and the HDFS DataNode.
 They may occur when establishing a connection, attempting to read, or attempting to write.


[31/37] hbase git commit: HBASE-15215 TestBlockEvictionFromClient is flaky in jdk1.7 build (setting offheap)

Posted by sy...@apache.org.
HBASE-15215 TestBlockEvictionFromClient is flaky in jdk1.7 build (setting
offheap)


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

Branch: refs/heads/hbase-12439
Commit: 538815d82a62cbcc7aaccec0a3bc4e44cb925277
Parents: bf4fcc3
Author: ramkrishna <ra...@gmail.com>
Authored: Fri Feb 26 11:43:00 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Fri Feb 26 11:43:00 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/TestBlockEvictionFromClient.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/538815d8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index f4d668c..d3f718b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -102,7 +102,7 @@ public class TestBlockEvictionFromClient {
                                                         // tests
     conf.setInt("hbase.regionserver.handler.count", 20);
     conf.setInt("hbase.bucketcache.size", 400);
-    conf.setStrings("hbase.bucketcache.ioengine", "heap");
+    conf.setStrings("hbase.bucketcache.ioengine", "offheap");
     conf.setFloat("hfile.block.cache.size", 0.2f);
     conf.setFloat("hbase.regionserver.global.memstore.size", 0.1f);
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);// do not retry


[28/37] hbase git commit: Revert "HBASE-15128 Disable region splits and merges switch in master"

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
index 9805d50..8dbb5ad 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/SnapshotProtos.java
@@ -11,13 +11,13 @@ public final class SnapshotProtos {
   public interface SnapshotFileInfoOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .hbase.pb.SnapshotFileInfo.Type type = 1;
+    // required .SnapshotFileInfo.Type type = 1;
     /**
-     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .SnapshotFileInfo.Type type = 1;</code>
      */
     boolean hasType();
     /**
-     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .SnapshotFileInfo.Type type = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType();
 
@@ -67,7 +67,7 @@ public final class SnapshotProtos {
         getWalNameBytes();
   }
   /**
-   * Protobuf type {@code hbase.pb.SnapshotFileInfo}
+   * Protobuf type {@code SnapshotFileInfo}
    */
   public static final class SnapshotFileInfo extends
       com.google.protobuf.GeneratedMessage
@@ -157,12 +157,12 @@ public final class SnapshotProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Builder.class);
     }
@@ -183,7 +183,7 @@ public final class SnapshotProtos {
     }
 
     /**
-     * Protobuf enum {@code hbase.pb.SnapshotFileInfo.Type}
+     * Protobuf enum {@code SnapshotFileInfo.Type}
      */
     public enum Type
         implements com.google.protobuf.ProtocolMessageEnum {
@@ -261,21 +261,21 @@ public final class SnapshotProtos {
         this.value = value;
       }
 
-      // @@protoc_insertion_point(enum_scope:hbase.pb.SnapshotFileInfo.Type)
+      // @@protoc_insertion_point(enum_scope:SnapshotFileInfo.Type)
     }
 
     private int bitField0_;
-    // required .hbase.pb.SnapshotFileInfo.Type type = 1;
+    // required .SnapshotFileInfo.Type type = 1;
     public static final int TYPE_FIELD_NUMBER = 1;
     private org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type type_;
     /**
-     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .SnapshotFileInfo.Type type = 1;</code>
      */
     public boolean hasType() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+     * <code>required .SnapshotFileInfo.Type type = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType() {
       return type_;
@@ -613,19 +613,19 @@ public final class SnapshotProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.SnapshotFileInfo}
+     * Protobuf type {@code SnapshotFileInfo}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
        implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfoOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Builder.class);
       }
@@ -667,7 +667,7 @@ public final class SnapshotProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotFileInfo_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotFileInfo_descriptor;
       }
 
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo getDefaultInstanceForType() {
@@ -767,22 +767,22 @@ public final class SnapshotProtos {
       }
       private int bitField0_;
 
-      // required .hbase.pb.SnapshotFileInfo.Type type = 1;
+      // required .SnapshotFileInfo.Type type = 1;
       private org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type type_ = org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type.HFILE;
       /**
-       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .SnapshotFileInfo.Type type = 1;</code>
        */
       public boolean hasType() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .SnapshotFileInfo.Type type = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type getType() {
         return type_;
       }
       /**
-       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .SnapshotFileInfo.Type type = 1;</code>
        */
       public Builder setType(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo.Type value) {
         if (value == null) {
@@ -794,7 +794,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.SnapshotFileInfo.Type type = 1;</code>
+       * <code>required .SnapshotFileInfo.Type type = 1;</code>
        */
       public Builder clearType() {
         bitField0_ = (bitField0_ & ~0x00000001);
@@ -1025,7 +1025,7 @@ public final class SnapshotProtos {
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotFileInfo)
+      // @@protoc_insertion_point(builder_scope:SnapshotFileInfo)
     }
 
     static {
@@ -1033,7 +1033,7 @@ public final class SnapshotProtos {
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotFileInfo)
+    // @@protoc_insertion_point(class_scope:SnapshotFileInfo)
   }
 
   public interface SnapshotRegionManifestOrBuilder
@@ -1049,47 +1049,47 @@ public final class SnapshotProtos {
      */
     int getVersion();
 
-    // required .hbase.pb.RegionInfo region_info = 2;
+    // required .RegionInfo region_info = 2;
     /**
-     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>required .RegionInfo region_info = 2;</code>
      */
     boolean hasRegionInfo();
     /**
-     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>required .RegionInfo region_info = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo();
     /**
-     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>required .RegionInfo region_info = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder();
 
-    // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;
+    // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> 
         getFamilyFilesList();
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index);
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     int getFamilyFilesCount();
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> 
         getFamilyFilesOrBuilderList();
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder(
         int index);
   }
   /**
-   * Protobuf type {@code hbase.pb.SnapshotRegionManifest}
+   * Protobuf type {@code SnapshotRegionManifest}
    */
   public static final class SnapshotRegionManifest extends
       com.google.protobuf.GeneratedMessage
@@ -1182,12 +1182,12 @@ public final class SnapshotProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder.class);
     }
@@ -1225,17 +1225,17 @@ public final class SnapshotProtos {
       com.google.protobuf.ByteString
           getNameBytes();
 
-      // optional .hbase.pb.Reference reference = 2;
+      // optional .Reference reference = 2;
       /**
-       * <code>optional .hbase.pb.Reference reference = 2;</code>
+       * <code>optional .Reference reference = 2;</code>
        */
       boolean hasReference();
       /**
-       * <code>optional .hbase.pb.Reference reference = 2;</code>
+       * <code>optional .Reference reference = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference();
       /**
-       * <code>optional .hbase.pb.Reference reference = 2;</code>
+       * <code>optional .Reference reference = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder();
 
@@ -1258,7 +1258,7 @@ public final class SnapshotProtos {
       long getFileSize();
     }
     /**
-     * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile}
+     * Protobuf type {@code SnapshotRegionManifest.StoreFile}
      */
     public static final class StoreFile extends
         com.google.protobuf.GeneratedMessage
@@ -1345,12 +1345,12 @@ public final class SnapshotProtos {
       }
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder.class);
       }
@@ -1414,23 +1414,23 @@ public final class SnapshotProtos {
         }
       }
 
-      // optional .hbase.pb.Reference reference = 2;
+      // optional .Reference reference = 2;
       public static final int REFERENCE_FIELD_NUMBER = 2;
       private org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference reference_;
       /**
-       * <code>optional .hbase.pb.Reference reference = 2;</code>
+       * <code>optional .Reference reference = 2;</code>
        */
       public boolean hasReference() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional .hbase.pb.Reference reference = 2;</code>
+       * <code>optional .Reference reference = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference() {
         return reference_;
       }
       /**
-       * <code>optional .hbase.pb.Reference reference = 2;</code>
+       * <code>optional .Reference reference = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder() {
         return reference_;
@@ -1652,19 +1652,19 @@ public final class SnapshotProtos {
         return builder;
       }
       /**
-       * Protobuf type {@code hbase.pb.SnapshotRegionManifest.StoreFile}
+       * Protobuf type {@code SnapshotRegionManifest.StoreFile}
        */
       public static final class Builder extends
           com.google.protobuf.GeneratedMessage.Builder<Builder>
          implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder {
         public static final com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor;
         }
 
         protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
             internalGetFieldAccessorTable() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
                   org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder.class);
         }
@@ -1709,7 +1709,7 @@ public final class SnapshotProtos {
 
         public com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_StoreFile_descriptor;
         }
 
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getDefaultInstanceForType() {
@@ -1882,18 +1882,18 @@ public final class SnapshotProtos {
           return this;
         }
 
-        // optional .hbase.pb.Reference reference = 2;
+        // optional .Reference reference = 2;
         private org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference reference_ = org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.getDefaultInstance();
         private com.google.protobuf.SingleFieldBuilder<
             org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder> referenceBuilder_;
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public boolean hasReference() {
           return ((bitField0_ & 0x00000002) == 0x00000002);
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference getReference() {
           if (referenceBuilder_ == null) {
@@ -1903,7 +1903,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public Builder setReference(org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference value) {
           if (referenceBuilder_ == null) {
@@ -1919,7 +1919,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public Builder setReference(
             org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder builderForValue) {
@@ -1933,7 +1933,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public Builder mergeReference(org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference value) {
           if (referenceBuilder_ == null) {
@@ -1952,7 +1952,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public Builder clearReference() {
           if (referenceBuilder_ == null) {
@@ -1965,7 +1965,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder getReferenceBuilder() {
           bitField0_ |= 0x00000002;
@@ -1973,7 +1973,7 @@ public final class SnapshotProtos {
           return getReferenceFieldBuilder().getBuilder();
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder getReferenceOrBuilder() {
           if (referenceBuilder_ != null) {
@@ -1983,7 +1983,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>optional .hbase.pb.Reference reference = 2;</code>
+         * <code>optional .Reference reference = 2;</code>
          */
         private com.google.protobuf.SingleFieldBuilder<
             org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference, org.apache.hadoop.hbase.protobuf.generated.FSProtos.Reference.Builder, org.apache.hadoop.hbase.protobuf.generated.FSProtos.ReferenceOrBuilder> 
@@ -2048,7 +2048,7 @@ public final class SnapshotProtos {
           return this;
         }
 
-        // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest.StoreFile)
+        // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest.StoreFile)
       }
 
       static {
@@ -2056,7 +2056,7 @@ public final class SnapshotProtos {
         defaultInstance.initFields();
       }
 
-      // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest.StoreFile)
+      // @@protoc_insertion_point(class_scope:SnapshotRegionManifest.StoreFile)
     }
 
     public interface FamilyFilesOrBuilder
@@ -2072,33 +2072,33 @@ public final class SnapshotProtos {
        */
       com.google.protobuf.ByteString getFamilyName();
 
-      // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;
+      // repeated .SnapshotRegionManifest.StoreFile store_files = 2;
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> 
           getStoreFilesList();
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index);
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       int getStoreFilesCount();
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> 
           getStoreFilesOrBuilderList();
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder(
           int index);
     }
     /**
-     * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles}
+     * Protobuf type {@code SnapshotRegionManifest.FamilyFiles}
      */
     public static final class FamilyFiles extends
         com.google.protobuf.GeneratedMessage
@@ -2178,12 +2178,12 @@ public final class SnapshotProtos {
       }
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder.class);
       }
@@ -2220,36 +2220,36 @@ public final class SnapshotProtos {
         return familyName_;
       }
 
-      // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;
+      // repeated .SnapshotRegionManifest.StoreFile store_files = 2;
       public static final int STORE_FILES_FIELD_NUMBER = 2;
       private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> storeFiles_;
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> getStoreFilesList() {
         return storeFiles_;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> 
           getStoreFilesOrBuilderList() {
         return storeFiles_;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public int getStoreFilesCount() {
         return storeFiles_.size();
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index) {
         return storeFiles_.get(index);
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+       * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder(
           int index) {
@@ -2428,19 +2428,19 @@ public final class SnapshotProtos {
         return builder;
       }
       /**
-       * Protobuf type {@code hbase.pb.SnapshotRegionManifest.FamilyFiles}
+       * Protobuf type {@code SnapshotRegionManifest.FamilyFiles}
        */
       public static final class Builder extends
           com.google.protobuf.GeneratedMessage.Builder<Builder>
          implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder {
         public static final com.google.protobuf.Descriptors.Descriptor
             getDescriptor() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
         }
 
         protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
             internalGetFieldAccessorTable() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable
               .ensureFieldAccessorsInitialized(
                   org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder.class);
         }
@@ -2483,7 +2483,7 @@ public final class SnapshotProtos {
 
         public com.google.protobuf.Descriptors.Descriptor
             getDescriptorForType() {
-          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
+          return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
         }
 
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getDefaultInstanceForType() {
@@ -2633,7 +2633,7 @@ public final class SnapshotProtos {
           return this;
         }
 
-        // repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;
+        // repeated .SnapshotRegionManifest.StoreFile store_files = 2;
         private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> storeFiles_ =
           java.util.Collections.emptyList();
         private void ensureStoreFilesIsMutable() {
@@ -2647,7 +2647,7 @@ public final class SnapshotProtos {
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> storeFilesBuilder_;
 
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> getStoreFilesList() {
           if (storeFilesBuilder_ == null) {
@@ -2657,7 +2657,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public int getStoreFilesCount() {
           if (storeFilesBuilder_ == null) {
@@ -2667,7 +2667,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile getStoreFiles(int index) {
           if (storeFilesBuilder_ == null) {
@@ -2677,7 +2677,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder setStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) {
@@ -2694,7 +2694,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder setStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) {
@@ -2708,7 +2708,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) {
           if (storeFilesBuilder_ == null) {
@@ -2724,7 +2724,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile value) {
@@ -2741,7 +2741,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) {
@@ -2755,7 +2755,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addStoreFiles(
             int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder builderForValue) {
@@ -2769,7 +2769,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder addAllStoreFiles(
             java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile> values) {
@@ -2783,7 +2783,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder clearStoreFiles() {
           if (storeFilesBuilder_ == null) {
@@ -2796,7 +2796,7 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public Builder removeStoreFiles(int index) {
           if (storeFilesBuilder_ == null) {
@@ -2809,14 +2809,14 @@ public final class SnapshotProtos {
           return this;
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder getStoreFilesBuilder(
             int index) {
           return getStoreFilesFieldBuilder().getBuilder(index);
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder getStoreFilesOrBuilder(
             int index) {
@@ -2826,7 +2826,7 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFileOrBuilder> 
              getStoreFilesOrBuilderList() {
@@ -2837,14 +2837,14 @@ public final class SnapshotProtos {
           }
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder addStoreFilesBuilder() {
           return getStoreFilesFieldBuilder().addBuilder(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.getDefaultInstance());
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder addStoreFilesBuilder(
             int index) {
@@ -2852,7 +2852,7 @@ public final class SnapshotProtos {
               index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.getDefaultInstance());
         }
         /**
-         * <code>repeated .hbase.pb.SnapshotRegionManifest.StoreFile store_files = 2;</code>
+         * <code>repeated .SnapshotRegionManifest.StoreFile store_files = 2;</code>
          */
         public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.StoreFile.Builder> 
              getStoreFilesBuilderList() {
@@ -2873,7 +2873,7 @@ public final class SnapshotProtos {
           return storeFilesBuilder_;
         }
 
-        // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest.FamilyFiles)
+        // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest.FamilyFiles)
       }
 
       static {
@@ -2881,7 +2881,7 @@ public final class SnapshotProtos {
         defaultInstance.initFields();
       }
 
-      // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest.FamilyFiles)
+      // @@protoc_insertion_point(class_scope:SnapshotRegionManifest.FamilyFiles)
     }
 
     private int bitField0_;
@@ -2901,58 +2901,58 @@ public final class SnapshotProtos {
       return version_;
     }
 
-    // required .hbase.pb.RegionInfo region_info = 2;
+    // required .RegionInfo region_info = 2;
     public static final int REGION_INFO_FIELD_NUMBER = 2;
     private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_;
     /**
-     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>required .RegionInfo region_info = 2;</code>
      */
     public boolean hasRegionInfo() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>required .RegionInfo region_info = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
       return regionInfo_;
     }
     /**
-     * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>required .RegionInfo region_info = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
       return regionInfo_;
     }
 
-    // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;
+    // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;
     public static final int FAMILY_FILES_FIELD_NUMBER = 3;
     private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> familyFiles_;
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> getFamilyFilesList() {
       return familyFiles_;
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> 
         getFamilyFilesOrBuilderList() {
       return familyFiles_;
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public int getFamilyFilesCount() {
       return familyFiles_.size();
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index) {
       return familyFiles_.get(index);
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+     * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder(
         int index) {
@@ -3152,19 +3152,19 @@ public final class SnapshotProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.SnapshotRegionManifest}
+     * Protobuf type {@code SnapshotRegionManifest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
        implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder.class);
       }
@@ -3214,7 +3214,7 @@ public final class SnapshotProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotRegionManifest_descriptor;
       }
 
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getDefaultInstanceForType() {
@@ -3376,18 +3376,18 @@ public final class SnapshotProtos {
         return this;
       }
 
-      // required .hbase.pb.RegionInfo region_info = 2;
+      // required .RegionInfo region_info = 2;
       private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public boolean hasRegionInfo() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
         if (regionInfoBuilder_ == null) {
@@ -3397,7 +3397,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public Builder setRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
         if (regionInfoBuilder_ == null) {
@@ -3413,7 +3413,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public Builder setRegionInfo(
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
@@ -3427,7 +3427,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public Builder mergeRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
         if (regionInfoBuilder_ == null) {
@@ -3446,7 +3446,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public Builder clearRegionInfo() {
         if (regionInfoBuilder_ == null) {
@@ -3459,7 +3459,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() {
         bitField0_ |= 0x00000002;
@@ -3467,7 +3467,7 @@ public final class SnapshotProtos {
         return getRegionInfoFieldBuilder().getBuilder();
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
         if (regionInfoBuilder_ != null) {
@@ -3477,7 +3477,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>required .RegionInfo region_info = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
@@ -3493,7 +3493,7 @@ public final class SnapshotProtos {
         return regionInfoBuilder_;
       }
 
-      // repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;
+      // repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;
       private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> familyFiles_ =
         java.util.Collections.emptyList();
       private void ensureFamilyFilesIsMutable() {
@@ -3507,7 +3507,7 @@ public final class SnapshotProtos {
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> familyFilesBuilder_;
 
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> getFamilyFilesList() {
         if (familyFilesBuilder_ == null) {
@@ -3517,7 +3517,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public int getFamilyFilesCount() {
         if (familyFilesBuilder_ == null) {
@@ -3527,7 +3527,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles getFamilyFiles(int index) {
         if (familyFilesBuilder_ == null) {
@@ -3537,7 +3537,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder setFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) {
@@ -3554,7 +3554,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder setFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) {
@@ -3568,7 +3568,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) {
         if (familyFilesBuilder_ == null) {
@@ -3584,7 +3584,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles value) {
@@ -3601,7 +3601,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) {
@@ -3615,7 +3615,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addFamilyFiles(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder builderForValue) {
@@ -3629,7 +3629,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder addAllFamilyFiles(
           java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles> values) {
@@ -3643,7 +3643,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder clearFamilyFiles() {
         if (familyFilesBuilder_ == null) {
@@ -3656,7 +3656,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public Builder removeFamilyFiles(int index) {
         if (familyFilesBuilder_ == null) {
@@ -3669,14 +3669,14 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder getFamilyFilesBuilder(
           int index) {
         return getFamilyFilesFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder getFamilyFilesOrBuilder(
           int index) {
@@ -3686,7 +3686,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFilesOrBuilder> 
            getFamilyFilesOrBuilderList() {
@@ -3697,14 +3697,14 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder addFamilyFilesBuilder() {
         return getFamilyFilesFieldBuilder().addBuilder(
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.getDefaultInstance());
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder addFamilyFilesBuilder(
           int index) {
@@ -3712,7 +3712,7 @@ public final class SnapshotProtos {
             index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.getDefaultInstance());
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
+       * <code>repeated .SnapshotRegionManifest.FamilyFiles family_files = 3;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.FamilyFiles.Builder> 
            getFamilyFilesBuilderList() {
@@ -3733,7 +3733,7 @@ public final class SnapshotProtos {
         return familyFilesBuilder_;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotRegionManifest)
+      // @@protoc_insertion_point(builder_scope:SnapshotRegionManifest)
     }
 
     static {
@@ -3741,53 +3741,53 @@ public final class SnapshotProtos {
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotRegionManifest)
+    // @@protoc_insertion_point(class_scope:SnapshotRegionManifest)
   }
 
   public interface SnapshotDataManifestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .hbase.pb.TableSchema table_schema = 1;
+    // required .TableSchema table_schema = 1;
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+     * <code>required .TableSchema table_schema = 1;</code>
      */
     boolean hasTableSchema();
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+     * <code>required .TableSchema table_schema = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema();
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+     * <code>required .TableSchema table_schema = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder();
 
-    // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;
+    // repeated .SnapshotRegionManifest region_manifests = 2;
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> 
         getRegionManifestsList();
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index);
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     int getRegionManifestsCount();
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> 
         getRegionManifestsOrBuilderList();
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder(
         int index);
   }
   /**
-   * Protobuf type {@code hbase.pb.SnapshotDataManifest}
+   * Protobuf type {@code SnapshotDataManifest}
    */
   public static final class SnapshotDataManifest extends
       com.google.protobuf.GeneratedMessage
@@ -3875,12 +3875,12 @@ public final class SnapshotProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
               org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.Builder.class);
     }
@@ -3901,58 +3901,58 @@ public final class SnapshotProtos {
     }
 
     private int bitField0_;
-    // required .hbase.pb.TableSchema table_schema = 1;
+    // required .TableSchema table_schema = 1;
     public static final int TABLE_SCHEMA_FIELD_NUMBER = 1;
     private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_;
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+     * <code>required .TableSchema table_schema = 1;</code>
      */
     public boolean hasTableSchema() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+     * <code>required .TableSchema table_schema = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
       return tableSchema_;
     }
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+     * <code>required .TableSchema table_schema = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
       return tableSchema_;
     }
 
-    // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;
+    // repeated .SnapshotRegionManifest region_manifests = 2;
     public static final int REGION_MANIFESTS_FIELD_NUMBER = 2;
     private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> regionManifests_;
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> getRegionManifestsList() {
       return regionManifests_;
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> 
         getRegionManifestsOrBuilderList() {
       return regionManifests_;
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     public int getRegionManifestsCount() {
       return regionManifests_.size();
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index) {
       return regionManifests_.get(index);
     }
     /**
-     * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+     * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder(
         int index) {
@@ -4135,19 +4135,19 @@ public final class SnapshotProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.SnapshotDataManifest}
+     * Protobuf type {@code SnapshotDataManifest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
        implements org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
                 org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.class, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest.Builder.class);
       }
@@ -4195,7 +4195,7 @@ public final class SnapshotProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_hbase_pb_SnapshotDataManifest_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.internal_static_SnapshotDataManifest_descriptor;
       }
 
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest getDefaultInstanceForType() {
@@ -4317,18 +4317,18 @@ public final class SnapshotProtos {
       }
       private int bitField0_;
 
-      // required .hbase.pb.TableSchema table_schema = 1;
+      // required .TableSchema table_schema = 1;
       private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public boolean hasTableSchema() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
         if (tableSchemaBuilder_ == null) {
@@ -4338,7 +4338,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public Builder setTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
         if (tableSchemaBuilder_ == null) {
@@ -4354,7 +4354,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public Builder setTableSchema(
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
@@ -4368,7 +4368,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public Builder mergeTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
         if (tableSchemaBuilder_ == null) {
@@ -4387,7 +4387,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public Builder clearTableSchema() {
         if (tableSchemaBuilder_ == null) {
@@ -4400,7 +4400,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getTableSchemaBuilder() {
         bitField0_ |= 0x00000001;
@@ -4408,7 +4408,7 @@ public final class SnapshotProtos {
         return getTableSchemaFieldBuilder().getBuilder();
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
         if (tableSchemaBuilder_ != null) {
@@ -4418,7 +4418,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>required .hbase.pb.TableSchema table_schema = 1;</code>
+       * <code>required .TableSchema table_schema = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
@@ -4434,7 +4434,7 @@ public final class SnapshotProtos {
         return tableSchemaBuilder_;
       }
 
-      // repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;
+      // repeated .SnapshotRegionManifest region_manifests = 2;
       private java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> regionManifests_ =
         java.util.Collections.emptyList();
       private void ensureRegionManifestsIsMutable() {
@@ -4448,7 +4448,7 @@ public final class SnapshotProtos {
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> regionManifestsBuilder_;
 
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> getRegionManifestsList() {
         if (regionManifestsBuilder_ == null) {
@@ -4458,7 +4458,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public int getRegionManifestsCount() {
         if (regionManifestsBuilder_ == null) {
@@ -4468,7 +4468,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest getRegionManifests(int index) {
         if (regionManifestsBuilder_ == null) {
@@ -4478,7 +4478,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder setRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) {
@@ -4495,7 +4495,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder setRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) {
@@ -4509,7 +4509,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) {
         if (regionManifestsBuilder_ == null) {
@@ -4525,7 +4525,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest value) {
@@ -4542,7 +4542,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(
           org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) {
@@ -4556,7 +4556,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addRegionManifests(
           int index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder builderForValue) {
@@ -4570,7 +4570,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder addAllRegionManifests(
           java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest> values) {
@@ -4584,7 +4584,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder clearRegionManifests() {
         if (regionManifestsBuilder_ == null) {
@@ -4597,7 +4597,7 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public Builder removeRegionManifests(int index) {
         if (regionManifestsBuilder_ == null) {
@@ -4610,14 +4610,14 @@ public final class SnapshotProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder getRegionManifestsBuilder(
           int index) {
         return getRegionManifestsFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder getRegionManifestsOrBuilder(
           int index) {
@@ -4627,7 +4627,7 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifestOrBuilder> 
            getRegionManifestsOrBuilderList() {
@@ -4638,14 +4638,14 @@ public final class SnapshotProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder addRegionManifestsBuilder() {
         return getRegionManifestsFieldBuilder().addBuilder(
             org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.getDefaultInstance());
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder addRegionManifestsBuilder(
           int index) {
@@ -4653,7 +4653,7 @@ public final class SnapshotProtos {
             index, org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.getDefaultInstance());
       }
       /**
-       * <code>repeated .hbase.pb.SnapshotRegionManifest region_manifests = 2;</code>
+       * <code>repeated .SnapshotRegionManifest region_manifests = 2;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest.Builder> 
            getRegionManifestsBuilderList() {
@@ -4674,7 +4674,7 @@ public final class SnapshotProtos {
         return regionManifestsBuilder_;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotDataManifest)
+      // @@protoc_insertion_point(builder_scope:SnapshotDataManifest)
     }
 
     static {
@@ -4682,34 +4682,34 @@ public final class SnapshotProtos {
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotDataManifest)
+    // @@protoc_insertion_point(class_scope:SnapshotDataManifest)
   }
 
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SnapshotFileInfo_descriptor;
+    internal_static_SnapshotFileInfo_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable;
+      internal_static_SnapshotFileInfo_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SnapshotRegionManifest_descriptor;
+    internal_static_SnapshotRegionManifest_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable;
+      internal_static_SnapshotRegionManifest_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor;
+    internal_static_SnapshotRegionManifest_StoreFile_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable;
+      internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor;
+    internal_static_SnapshotRegionManifest_FamilyFiles_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable;
+      internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SnapshotDataManifest_descriptor;
+    internal_static_SnapshotDataManifest_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_SnapshotDataManifest_fieldAccessorTable;
+      internal_static_SnapshotDataManifest_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -4719,60 +4719,58 @@ public final class SnapshotProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\016Snapshot.proto\022\010hbase.pb\032\010FS.proto\032\013HB" +
-      "ase.proto\"\222\001\n\020SnapshotFileInfo\022-\n\004type\030\001" +
-      " \002(\0162\037.hbase.pb.SnapshotFileInfo.Type\022\r\n" +
-      "\005hfile\030\003 \001(\t\022\022\n\nwal_server\030\004 \001(\t\022\020\n\010wal_" +
-      "name\030\005 \001(\t\"\032\n\004Type\022\t\n\005HFILE\020\001\022\007\n\003WAL\020\002\"\323" +
-      "\002\n\026SnapshotRegionManifest\022\017\n\007version\030\001 \001" +
-      "(\005\022)\n\013region_info\030\002 \002(\0132\024.hbase.pb.Regio" +
-      "nInfo\022B\n\014family_files\030\003 \003(\0132,.hbase.pb.S" +
-      "napshotRegionManifest.FamilyFiles\032T\n\tSto" +
-      "reFile\022\014\n\004name\030\001 \002(\t\022&\n\treference\030\002 \001(\0132",
-      "\023.hbase.pb.Reference\022\021\n\tfile_size\030\003 \001(\004\032" +
-      "c\n\013FamilyFiles\022\023\n\013family_name\030\001 \002(\014\022?\n\013s" +
-      "tore_files\030\002 \003(\0132*.hbase.pb.SnapshotRegi" +
-      "onManifest.StoreFile\"\177\n\024SnapshotDataMani" +
-      "fest\022+\n\014table_schema\030\001 \002(\0132\025.hbase.pb.Ta" +
-      "bleSchema\022:\n\020region_manifests\030\002 \003(\0132 .hb" +
-      "ase.pb.SnapshotRegionManifestBD\n*org.apa" +
-      "che.hadoop.hbase.protobuf.generatedB\016Sna" +
-      "pshotProtosH\001\210\001\001\240\001\001"
+      "\n\016Snapshot.proto\032\010FS.proto\032\013HBase.proto\"" +
+      "\211\001\n\020SnapshotFileInfo\022$\n\004type\030\001 \002(\0162\026.Sna" +
+      "pshotFileInfo.Type\022\r\n\005hfile\030\003 \001(\t\022\022\n\nwal" +
+      "_server\030\004 \001(\t\022\020\n\010wal_name\030\005 \001(\t\"\032\n\004Type\022" +
+      "\t\n\005HFILE\020\001\022\007\n\003WAL\020\002\"\257\002\n\026SnapshotRegionMa" +
+      "nifest\022\017\n\007version\030\001 \001(\005\022 \n\013region_info\030\002" +
+      " \002(\0132\013.RegionInfo\0229\n\014family_files\030\003 \003(\0132" +
+      "#.SnapshotRegionManifest.FamilyFiles\032K\n\t" +
+      "StoreFile\022\014\n\004name\030\001 \002(\t\022\035\n\treference\030\002 \001" +
+      "(\0132\n.Reference\022\021\n\tfile_size\030\003 \001(\004\032Z\n\013Fam",
+      "ilyFiles\022\023\n\013family_name\030\001 \002(\014\0226\n\013store_f" +
+      "iles\030\002 \003(\0132!.SnapshotRegionManifest.Stor" +
+      "eFile\"m\n\024SnapshotDataManifest\022\"\n\014table_s" +
+      "chema\030\001 \002(\0132\014.TableSchema\0221\n\020region_mani" +
+      "fests\030\002 \003(\0132\027.SnapshotRegionManifestBD\n*" +
+      "org.apache.hadoop.hbase.protobuf.generat" +
+      "edB\016SnapshotProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
         public com.google.protobuf.ExtensionRegistry assignDescriptors(
             com.google.protobuf.Descriptors.FileDescriptor root) {
           descriptor = root;
-          internal_static_hbase_pb_SnapshotFileInfo_descriptor =
+          internal_static_SnapshotFileInfo_descriptor =
             getDescriptor().getMessageTypes().get(0);
-          internal_static_hbase_pb_SnapshotFileInfo_fieldAccessorTable = new
+          internal_static_SnapshotFileInfo_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_SnapshotFileInfo_descriptor,
+              internal_static_SnapshotFileInfo_descriptor,
               new java.lang.String[] { "Type", "Hfile", "WalServer", "WalName", });
-          internal_static_hbase_pb_SnapshotRegionManifest_descriptor =
+          internal_static_SnapshotRegionManifest_descriptor =
             getDescriptor().getMessageTypes().get(1);
-          internal_static_hbase_pb_SnapshotRegionManifest_fieldAccessorTable = new
+          internal_static_SnapshotRegionManifest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_SnapshotRegionManifest_descriptor,
+              internal_static_SnapshotRegionManifest_descriptor,
               new java.lang.String[] { "Version", "RegionInfo", "FamilyFiles", });
-          internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor =
-            internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(0);
-          internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_fieldAccessorTable = new
+          internal_static_SnapshotRegionManifest_StoreFile_descriptor =
+            internal_static_SnapshotRegionManifest_descriptor.getNestedTypes().get(0);
+          internal_static_SnapshotRegionManifest_StoreFile_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_SnapshotRegionManifest_StoreFile_descriptor,
+              internal_static_SnapshotRegionManifest_StoreFile_descriptor,
               new java.lang.String[] { "Name", "Reference", "FileSize", });
-          internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor =
-            internal_static_hbase_pb_SnapshotRegionManifest_descriptor.getNestedTypes().get(1);
-          internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable = new
+          internal_static_SnapshotRegionManifest_FamilyFiles_descriptor =
+            internal_static_SnapshotRegionManifest_descriptor.getNestedTypes().get(1);
+          internal_static_SnapshotRegionManifest_FamilyFiles_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_SnapshotRegionManifest_FamilyFiles_descriptor,
+              internal_static_SnapshotRegionManifest_FamilyFiles_descriptor,
            

<TRUNCATED>

[13/37] hbase git commit: HBASE-15312 Update the dependences of pom for mini cluster in HBase Book (Liu Shaohui)

Posted by sy...@apache.org.
HBASE-15312 Update the dependences of pom for mini cluster in HBase Book (Liu Shaohui)


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

Branch: refs/heads/hbase-12439
Commit: 2a306437aaccb99ff333ab41c7165333994eba48
Parents: 30cec72
Author: stack <st...@apache.org>
Authored: Wed Feb 24 09:31:10 2016 -0800
Committer: stack <st...@apache.org>
Committed: Wed Feb 24 09:31:10 2016 -0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/unit_testing.adoc | 60 ++++++++++++++--------
 1 file changed, 39 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2a306437/src/main/asciidoc/_chapters/unit_testing.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/unit_testing.adoc b/src/main/asciidoc/_chapters/unit_testing.adoc
index e1bcf87..15b6cce 100644
--- a/src/main/asciidoc/_chapters/unit_testing.adoc
+++ b/src/main/asciidoc/_chapters/unit_testing.adoc
@@ -268,37 +268,55 @@ Check the versions to be sure they are appropriate.
 
 [source,xml]
 ----
+<properties>
+  <hbase.version>2.0.0-SNAPSHOT</hbase.version>
+  <hadoop.version>2.7.1</hadoop.version>
+</properties>
 
-<dependency>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-common</artifactId>
-    <version>2.0.0</version>
+<dependencies>
+  <dependency>
+    <groupId>org.apache.hbase</groupId>
+    <artifactId>hbase-server</artifactId>
+    <version>${hbase.version}</version>
+    <scope>test</scope>
+  </dependency>
+  <dependency>
+    <groupId>org.apache.hbase</groupId>
+    <artifactId>hbase-server</artifactId>
+    <version>${hbase.version}</version>
     <type>test-jar</type>
     <scope>test</scope>
-</dependency>
-
-<dependency>
+  </dependency>
+  <dependency>
     <groupId>org.apache.hbase</groupId>
-    <artifactId>hbase</artifactId>
-    <version>0.98.3</version>
+    <artifactId>hbase-hadoop-compat</artifactId>
+    <version>${hbase.version}</version>
     <type>test-jar</type>
     <scope>test</scope>
-</dependency>
+  </dependency>
 
-<dependency>
+  <dependency>
     <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-hdfs</artifactId>
-    <version>2.0.0</version>
+    <artifactId>hadoop-common</artifactId>
+    <version>${hadoop.version}</version>
     <type>test-jar</type>
     <scope>test</scope>
-</dependency>
-
-<dependency>
+  </dependency>
+  <dependency>
+    <groupId>org.apache.hbase</groupId>
+    <artifactId>hbase-hadoop2-compat</artifactId>
+    <version>${hbase.version}</version>
+    <type>test-jar</type>
+    <scope>test</scope>
+  </dependency>
+  <dependency>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-hdfs</artifactId>
-    <version>2.0.0</version>
+    <version>${hadoop.version}</version>
+    <type>test-jar</type>
     <scope>test</scope>
-</dependency>
+  </dependency>
+</dependencies>
 ----
 
 This code represents an integration test for the MyDAO insert shown in <<unit.tests,unit.tests>>.
@@ -309,7 +327,8 @@ This code represents an integration test for the MyDAO insert shown in <<unit.te
 public class MyHBaseIntegrationTest {
     private static HBaseTestingUtility utility;
     byte[] CF = "CF".getBytes();
-    byte[] QUALIFIER = "CQ-1".getBytes();
+    byte[] CQ1 = "CQ-1".getBytes();
+    byte[] CQ2 = "CQ-2".getBytes();
 
     @Before
     public void setup() throws Exception {
@@ -319,8 +338,7 @@ public class MyHBaseIntegrationTest {
 
     @Test
         public void testInsert() throws Exception {
-       	 HTableInterface table = utility.createTable(Bytes.toBytes("MyTest"),
-       			 Bytes.toBytes("CF"));
+       	 HTableInterface table = utility.createTable(Bytes.toBytes("MyTest"), CF);
        	 HBaseTestObj obj = new HBaseTestObj();
        	 obj.setRowKey("ROWKEY-1");
        	 obj.setData1("DATA-1");


[27/37] hbase git commit: Revert "HBASE-15128 Disable region splits and merges switch in master"

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
index 0240a67..4371739 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java
@@ -8196,450 +8196,6 @@ public final class ZooKeeperProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.TableLock)
   }
 
-  public interface SwitchStateOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // optional bool enabled = 1;
-    /**
-     * <code>optional bool enabled = 1;</code>
-     */
-    boolean hasEnabled();
-    /**
-     * <code>optional bool enabled = 1;</code>
-     */
-    boolean getEnabled();
-  }
-  /**
-   * Protobuf type {@code hbase.pb.SwitchState}
-   *
-   * <pre>
-   **
-   * State of the switch.
-   * </pre>
-   */
-  public static final class SwitchState extends
-      com.google.protobuf.GeneratedMessage
-      implements SwitchStateOrBuilder {
-    // Use SwitchState.newBuilder() to construct.
-    private SwitchState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private SwitchState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final SwitchState defaultInstance;
-    public static SwitchState getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public SwitchState getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private SwitchState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              enabled_ = input.readBool();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<SwitchState> PARSER =
-        new com.google.protobuf.AbstractParser<SwitchState>() {
-      public SwitchState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new SwitchState(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<SwitchState> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // optional bool enabled = 1;
-    public static final int ENABLED_FIELD_NUMBER = 1;
-    private boolean enabled_;
-    /**
-     * <code>optional bool enabled = 1;</code>
-     */
-    public boolean hasEnabled() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional bool enabled = 1;</code>
-     */
-    public boolean getEnabled() {
-      return enabled_;
-    }
-
-    private void initFields() {
-      enabled_ = false;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBool(1, enabled_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(1, enabled_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState) obj;
-
-      boolean result = true;
-      result = result && (hasEnabled() == other.hasEnabled());
-      if (hasEnabled()) {
-        result = result && (getEnabled()
-            == other.getEnabled());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasEnabled()) {
-        hash = (37 * hash) + ENABLED_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getEnabled());
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.SwitchState}
-     *
-     * <pre>
-     **
-     * State of the switch.
-     * </pre>
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        enabled_ = false;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_SwitchState_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState build() {
-        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.enabled_ = enabled_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState.getDefaultInstance()) return this;
-        if (other.hasEnabled()) {
-          setEnabled(other.getEnabled());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // optional bool enabled = 1;
-      private boolean enabled_ ;
-      /**
-       * <code>optional bool enabled = 1;</code>
-       */
-      public boolean hasEnabled() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional bool enabled = 1;</code>
-       */
-      public boolean getEnabled() {
-        return enabled_;
-      }
-      /**
-       * <code>optional bool enabled = 1;</code>
-       */
-      public Builder setEnabled(boolean value) {
-        bitField0_ |= 0x00000001;
-        enabled_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool enabled = 1;</code>
-       */
-      public Builder clearEnabled() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        enabled_ = false;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SwitchState)
-    }
-
-    static {
-      defaultInstance = new SwitchState(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.SwitchState)
-  }
-
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_MetaRegionServer_descriptor;
   private static
@@ -8690,11 +8246,6 @@ public final class ZooKeeperProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_TableLock_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SwitchState_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_hbase_pb_SwitchState_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -8735,10 +8286,9 @@ public final class ZooKeeperProtos {
       "\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableNam" +
       "e\022(\n\nlock_owner\030\002 \001(\0132\024.hbase.pb.ServerN",
       "ame\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(" +
-      "\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003\"" +
-      "\036\n\013SwitchState\022\017\n\007enabled\030\001 \001(\010BE\n*org.a" +
-      "pache.hadoop.hbase.protobuf.generatedB\017Z" +
-      "ooKeeperProtosH\001\210\001\001\240\001\001"
+      "\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003B" +
+      "E\n*org.apache.hadoop.hbase.protobuf.gene" +
+      "ratedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8805,12 +8355,6 @@ public final class ZooKeeperProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_TableLock_descriptor,
               new java.lang.String[] { "TableName", "LockOwner", "ThreadId", "IsShared", "Purpose", "CreateTime", });
-          internal_static_hbase_pb_SwitchState_descriptor =
-            getDescriptor().getMessageTypes().get(10);
-          internal_static_hbase_pb_SwitchState_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_hbase_pb_SwitchState_descriptor,
-              new java.lang.String[] { "Enabled", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index 79bb862..aa31a5e 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -279,29 +279,6 @@ message IsBalancerEnabledResponse {
   required bool enabled = 1;
 }
 
-enum MasterSwitchType {
-  SPLIT = 0;
-  MERGE = 1;
-}
-
-message SetSplitOrMergeEnabledRequest {
-  required bool enabled = 1;
-  optional bool synchronous = 2;
-  repeated MasterSwitchType switch_types = 3;
-}
-
-message SetSplitOrMergeEnabledResponse {
-  repeated bool prev_value = 1;
-}
-
-message IsSplitOrMergeEnabledRequest {
-  required MasterSwitchType switch_type = 1;
-}
-
-message IsSplitOrMergeEnabledResponse {
-  required bool enabled = 1;
-}
-
 message NormalizeRequest {
 }
 
@@ -656,19 +633,6 @@ service MasterService {
     returns(IsBalancerEnabledResponse);
 
   /**
-   * Turn the split or merge switch on or off.
-   * If synchronous is true, it waits until current operation call, if outstanding, to return.
-   */
-  rpc SetSplitOrMergeEnabled(SetSplitOrMergeEnabledRequest)
-    returns(SetSplitOrMergeEnabledResponse);
-
-  /**
-   * Query whether the split or merge switch is on/off.
-   */
-  rpc IsSplitOrMergeEnabled(IsSplitOrMergeEnabledRequest)
-    returns(IsSplitOrMergeEnabledResponse);
-
-  /**
    * Run region normalizer. Can NOT run for various reasons. Check logs.
    */
   rpc Normalize(NormalizeRequest)

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-protocol/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
index 4963c09..54652af 100644
--- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol/src/main/protobuf/ZooKeeper.proto
@@ -153,10 +153,3 @@ message TableLock {
   optional string purpose = 5;
   optional int64 create_time = 6;
 }
-
-/**
- * State of the switch.
- */
-message SwitchState {
-  optional bool enabled = 1;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 1110db3..53a080e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.RegionStateListener;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
@@ -2354,11 +2353,6 @@ public class AssignmentManager {
       return hri.getShortNameToLog() + " is not opening on " + serverName;
     }
 
-    if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
-            Admin.MasterSwitchType.SPLIT)) {
-      return "split switch is off!";
-    }
-
     // Just return in case of retrying
     if (current.isSplitting()) {
       return null;
@@ -2517,10 +2511,6 @@ public class AssignmentManager {
       return "Merging daughter region already exists, p=" + current;
     }
 
-    if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
-            Admin.MasterSwitchType.MERGE)) {
-      return "merge switch is off!";
-    }
     // Just return in case of retrying
     if (current != null) {
       return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 6806c2d..5d8c325 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -76,7 +76,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
@@ -156,7 +155,6 @@ import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
-import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTrackerManager;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -255,9 +253,6 @@ public class HMaster extends HRegionServer implements MasterServices {
   // Tracker for load balancer state
   LoadBalancerTracker loadBalancerTracker;
 
-  // Tracker for split and merge state
-  SplitOrMergeTrackerManager splitOrMergeTracker;
-
   // Tracker for region normalizer state
   private RegionNormalizerTracker regionNormalizerTracker;
 
@@ -583,13 +578,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.normalizer.setMasterServices(this);
     this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
     this.loadBalancerTracker.start();
-
     this.regionNormalizerTracker = new RegionNormalizerTracker(zooKeeper, this);
     this.regionNormalizerTracker.start();
-
-    this.splitOrMergeTracker = new SplitOrMergeTrackerManager(zooKeeper, conf, this);
-    this.splitOrMergeTracker.start();
-
     this.assignmentManager = new AssignmentManager(this, serverManager,
       this.balancer, this.service, this.metricsMaster,
       this.tableLockManager, tableStateManager);
@@ -2793,20 +2783,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     return null == regionNormalizerTracker? false: regionNormalizerTracker.isNormalizerOn();
   }
 
-
-  /**
-   * Queries the state of the {@link SplitOrMergeTrackerManager}. If it is not initialized,
-   * false is returned. If switchType is illegal, false will return.
-   * @param switchType see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}
-   * @return The state of the switch
-   */
-  public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) {
-    if (null == splitOrMergeTracker) {
-      return false;
-    }
-    return splitOrMergeTracker.isSplitOrMergeEnabled(switchType);
-  }
-
   /**
    * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
    *
@@ -2823,8 +2799,4 @@ public class HMaster extends HRegionServer implements MasterServices {
   public RegionNormalizerTracker getRegionNormalizerTracker() {
     return regionNormalizerTracker;
   }
-
-  public SplitOrMergeTrackerManager getSplitOrMergeTracker() {
-    return splitOrMergeTracker;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 1c770d1..1dd4c14 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
@@ -1507,35 +1506,6 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
-  public SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(RpcController controller,
-    SetSplitOrMergeEnabledRequest request) throws ServiceException {
-    SetSplitOrMergeEnabledResponse.Builder response = SetSplitOrMergeEnabledResponse.newBuilder();
-    try {
-      master.checkInitialized();
-      for (MasterSwitchType masterSwitchType : request.getSwitchTypesList()) {
-        Admin.MasterSwitchType switchType = convert(masterSwitchType);
-        boolean oldValue = master.isSplitOrMergeEnabled(switchType);
-        boolean newValue = request.getEnabled();
-        master.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue, switchType);
-        response.addPrevValue(oldValue);
-      }
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    } catch (KeeperException e) {
-      throw new ServiceException(e);
-    }
-    return response.build();
-  }
-
-  @Override
-  public IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(RpcController controller,
-    IsSplitOrMergeEnabledRequest request) throws ServiceException {
-    IsSplitOrMergeEnabledResponse.Builder response = IsSplitOrMergeEnabledResponse.newBuilder();
-    response.setEnabled(master.isSplitOrMergeEnabled(convert(request.getSwitchType())));
-    return response.build();
-  }
-
-  @Override
   public NormalizeResponse normalize(RpcController controller,
       NormalizeRequest request) throws ServiceException {
     try {
@@ -1604,16 +1574,4 @@ public class MasterRpcServices extends RSRpcServices
     }
     return response.build();
   }
-
-  private Admin.MasterSwitchType convert(MasterSwitchType switchType) {
-    switch (switchType) {
-      case SPLIT:
-        return Admin.MasterSwitchType.SPLIT;
-      case MERGE:
-        return Admin.MasterSwitchType.MERGE;
-      default:
-        break;
-    }
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 93287ad..a9113ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -235,7 +235,6 @@ public class HBaseFsck extends Configured implements Closeable {
   private long timelag = DEFAULT_TIME_LAG; // tables whose modtime is older
   private static boolean forceExclusive = false; // only this hbck can modify HBase
   private static boolean disableBalancer = false; // disable load balancer to keep regions stable
-  private static boolean disableSplitAndMerge = false; // disable split and merge
   private boolean fixAssignments = false; // fix assignment errors?
   private boolean fixMeta = false; // fix meta errors?
   private boolean checkHdfs = true; // load and check fs consistency?
@@ -684,11 +683,6 @@ public class HBaseFsck extends Configured implements Closeable {
     if (shouldDisableBalancer()) {
       oldBalancer = admin.setBalancerRunning(false, true);
     }
-    boolean[] oldSplitAndMerge = null;
-    if (shouldDisableSplitAndMerge()) {
-      oldSplitAndMerge = admin.setSplitOrMergeEnabled(false, false,
-        Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
-    }
 
     try {
       onlineConsistencyRepair();
@@ -700,17 +694,6 @@ public class HBaseFsck extends Configured implements Closeable {
       if (shouldDisableBalancer() && oldBalancer) {
         admin.setBalancerRunning(oldBalancer, false);
       }
-
-      if (shouldDisableSplitAndMerge()) {
-        if (oldSplitAndMerge != null) {
-          if (oldSplitAndMerge[0]) {
-            admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
-          }
-          if (oldSplitAndMerge[1]) {
-            admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
-          }
-        }
-      }
     }
 
     if (checkRegionBoundaries) {
@@ -4201,13 +4184,6 @@ public class HBaseFsck extends Configured implements Closeable {
   }
 
   /**
-   * Disable the split and merge
-   */
-  public static void setDisableSplitAndMerge() {
-    disableSplitAndMerge = true;
-  }
-
-  /**
    * The balancer should be disabled if we are modifying HBase.
    * It can be disabled if you want to prevent region movement from causing
    * false positives.
@@ -4217,15 +4193,6 @@ public class HBaseFsck extends Configured implements Closeable {
   }
 
   /**
-   * The split and merge should be disabled if we are modifying HBase.
-   * It can be disabled if you want to prevent region movement from causing
-   * false positives.
-   */
-  public boolean shouldDisableSplitAndMerge() {
-    return fixAny || disableSplitAndMerge;
-  }
-
-  /**
    * Set summary mode.
    * Print only summary of the tables and status (OK or INCONSISTENT)
    */
@@ -4584,8 +4551,6 @@ public class HBaseFsck extends Configured implements Closeable {
         setForceExclusive();
       } else if (cmd.equals("-disableBalancer")) {
         setDisableBalancer();
-      }  else if (cmd.equals("-disableSplitAndMerge")) {
-        setDisableSplitAndMerge();
       } else if (cmd.equals("-timelag")) {
         if (i == args.length - 1) {
           errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -timelag needs a value.");

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java
deleted file mode 100644
index 1495dd1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTrackerManager.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-
-
-/**
- * Tracks the switch of split and merge states in ZK
- *
- */
-@InterfaceAudience.Private
-public class SplitOrMergeTrackerManager {
-
-  private String splitZnode;
-  private String mergeZnode;
-
-  private SwitchStateTracker splitStateTracker;
-  private SwitchStateTracker mergeStateTracker;
-
-  public SplitOrMergeTrackerManager(ZooKeeperWatcher watcher, Configuration conf,
-                                    Abortable abortable) {
-    try {
-      if (ZKUtil.checkExists(watcher, watcher.getSwitchZNode()) < 0) {
-        ZKUtil.createAndFailSilent(watcher, watcher.getSwitchZNode());
-      }
-    } catch (KeeperException e) {
-      throw new RuntimeException(e);
-    }
-    splitZnode = ZKUtil.joinZNode(watcher.getSwitchZNode(),
-      conf.get("zookeeper.znode.switch.split", "split"));
-    mergeZnode = ZKUtil.joinZNode(watcher.getSwitchZNode(),
-      conf.get("zookeeper.znode.switch.merge", "merge"));
-    splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
-    mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);
-  }
-
-  public void start() {
-    splitStateTracker.start();
-    mergeStateTracker.start();
-  }
-
-  public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) {
-    switch (switchType) {
-      case SPLIT:
-        return splitStateTracker.isSwitchEnabled();
-      case MERGE:
-        return mergeStateTracker.isSwitchEnabled();
-      default:
-        break;
-    }
-    return false;
-  }
-
-  public void setSplitOrMergeEnabled(boolean enabled, Admin.MasterSwitchType switchType)
-    throws KeeperException {
-    switch (switchType) {
-      case SPLIT:
-        splitStateTracker.setSwitchEnabled(enabled);
-        break;
-      case MERGE:
-        mergeStateTracker.setSwitchEnabled(enabled);
-        break;
-      default:
-        break;
-    }
-  }
-
-  private static class SwitchStateTracker extends ZooKeeperNodeTracker {
-
-    public SwitchStateTracker(ZooKeeperWatcher watcher, String node, Abortable abortable) {
-      super(watcher, node, abortable);
-    }
-
-    /**
-     * Return true if the switch is on, false otherwise
-     */
-    public boolean isSwitchEnabled() {
-      byte [] upData = super.getData(false);
-      try {
-        // if data in ZK is null, use default of on.
-        return upData == null || parseFrom(upData).getEnabled();
-      } catch (DeserializationException dex) {
-        LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData));
-        // return false to be safe.
-        return false;
-      }
-    }
-
-    /**
-     * Set the switch on/off
-     * @param enabled switch enabled or not?
-     * @throws KeeperException keepException will be thrown out
-     */
-    public void setSwitchEnabled(boolean enabled) throws KeeperException {
-      byte [] upData = toByteArray(enabled);
-      try {
-        ZKUtil.setData(watcher, node, upData);
-      } catch(KeeperException.NoNodeException nne) {
-        ZKUtil.createAndWatch(watcher, node, upData);
-      }
-      super.nodeDataChanged(node);
-    }
-
-    private byte [] toByteArray(boolean enabled) {
-      SwitchState.Builder builder = SwitchState.newBuilder();
-      builder.setEnabled(enabled);
-      return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
-    }
-
-    private SwitchState parseFrom(byte [] bytes)
-      throws DeserializationException {
-      ProtobufUtil.expectPBMagicPrefix(bytes);
-      SwitchState.Builder builder = SwitchState.newBuilder();
-      try {
-        int magicLen = ProtobufUtil.lengthOfPBMagic();
-        ProtobufUtil.mergeFrom(builder, bytes, magicLen, bytes.length - magicLen);
-      } catch (IOException e) {
-        throw new DeserializationException(e);
-      }
-      return builder.build();
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
deleted file mode 100644
index 6405a14..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.client;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.IOException;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-@Category({MediumTests.class, ClientTests.class})
-public class TestSplitOrMergeStatus {
-
-  private static final Log LOG = LogFactory.getLog(TestSplitOrMergeStatus.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static byte [] FAMILY = Bytes.toBytes("testFamily");
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.startMiniCluster(2);
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testSplitSwitch() throws Exception {
-    TableName name = TableName.valueOf("testSplitSwitch");
-    Table t = TEST_UTIL.createTable(name, FAMILY);
-    TEST_UTIL.loadTable(t, FAMILY, false);
-
-    RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName());
-    int orignalCount = locator.getAllRegionLocations().size();
-
-    Admin admin = TEST_UTIL.getAdmin();
-    initSwitchStatus(admin);
-    boolean[] results = admin.setSplitOrMergeEnabled(false, false, Admin.MasterSwitchType.SPLIT);
-    assertEquals(results.length, 1);
-    assertTrue(results[0]);
-    admin.split(t.getName());
-    int count = waitOnSplitOrMerge(t).size();
-    assertTrue(orignalCount == count);
-
-    results = admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
-    assertEquals(results.length, 1);
-    assertFalse(results[0]);
-    admin.split(t.getName());
-    count = waitOnSplitOrMerge(t).size();
-    assertTrue(orignalCount<count);
-    admin.close();
-  }
-
-
-  @Test
-  public void testMergeSwitch() throws Exception {
-    TableName name = TableName.valueOf("testMergeSwitch");
-    Table t = TEST_UTIL.createTable(name, FAMILY);
-    TEST_UTIL.loadTable(t, FAMILY, false);
-
-    RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName());
-
-    Admin admin = TEST_UTIL.getAdmin();
-    initSwitchStatus(admin);
-    admin.split(t.getName());
-    waitOnSplitOrMerge(t); //Split the table to ensure we have two regions at least.
-
-    waitForMergable(admin, name);
-    int orignalCount = locator.getAllRegionLocations().size();
-    boolean[] results = admin.setSplitOrMergeEnabled(false, false, Admin.MasterSwitchType.MERGE);
-    assertEquals(results.length, 1);
-    assertTrue(results[0]);
-    List<HRegionInfo> regions = admin.getTableRegions(t.getName());
-    assertTrue(regions.size() > 1);
-    admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
-      regions.get(1).getEncodedNameAsBytes(), true);
-    int count = waitOnSplitOrMerge(t).size();
-    assertTrue(orignalCount == count);
-
-    waitForMergable(admin, name);
-    results = admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
-    assertEquals(results.length, 1);
-    assertFalse(results[0]);
-    admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
-      regions.get(1).getEncodedNameAsBytes(), true);
-    count = waitOnSplitOrMerge(t).size();
-    assertTrue(orignalCount>count);
-    admin.close();
-  }
-
-  @Test
-  public void testMultiSwitches() throws IOException {
-    Admin admin = TEST_UTIL.getAdmin();
-    boolean[] switches = admin.setSplitOrMergeEnabled(false, false,
-      Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
-    for (boolean s : switches){
-      assertTrue(s);
-    }
-    assertFalse(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT));
-    assertFalse(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE));
-    admin.close();
-  }
-
-  private void initSwitchStatus(Admin admin) throws IOException {
-    if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT)) {
-      admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
-    }
-    if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE)) {
-      admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
-    }
-    assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT));
-    assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE));
-  }
-
-  private void waitForMergable(Admin admin, TableName t) throws InterruptedException, IOException {
-    // Wait for the Regions to be mergeable
-    MiniHBaseCluster miniCluster = TEST_UTIL.getMiniHBaseCluster();
-    int mergeable = 0;
-    while (mergeable < 2) {
-      Thread.sleep(100);
-      admin.majorCompact(t);
-      mergeable = 0;
-      for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
-        for (Region region: regionThread.getRegionServer().getOnlineRegions(t)) {
-          mergeable += ((HRegion)region).isMergeable() ? 1 : 0;
-        }
-      }
-    }
-  }
-
-  /*
-   * Wait on table split.  May return because we waited long enough on the split
-   * and it didn't happen.  Caller should check.
-   * @param t
-   * @return Map of table regions; caller needs to check table actually split.
-   */
-  private List<HRegionLocation> waitOnSplitOrMerge(final Table t)
-    throws IOException {
-    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
-      List<HRegionLocation> regions = locator.getAllRegionLocations();
-      int originalCount = regions.size();
-      for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 10); i++) {
-        Thread.currentThread();
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          e.printStackTrace();
-        }
-        regions = locator.getAllRegionLocations();
-        if (regions.size() !=  originalCount)
-          break;
-      }
-      return regions;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 40c3711..82f0700 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -132,38 +132,6 @@ module Hbase
       end
     end
 
-    #----------------------------------------------------------------------------------------------
-    # Enable/disable one split or merge switch
-    # Returns previous switch setting.
-    def splitormerge_switch(type, enabled)
-      switch_type = nil
-      if type == 'SPLIT'
-        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::SPLIT
-      elsif type == 'MERGE'
-        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::MERGE
-      else
-        raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
-      end
-      @admin.setSplitOrMergeEnabled(
-        java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false),
-        switch_type)[0]
-    end
-
-    #----------------------------------------------------------------------------------------------
-    # Query the current state of the split or merge switch.
-    # Returns the switch's state (true is enabled).
-    def splitormerge_enabled(type)
-      switch_type = nil
-      if type == 'SPLIT'
-        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::SPLIT
-      elsif type == 'MERGE'
-        switch_type = org.apache.hadoop.hbase.client.Admin::MasterSwitchType::MERGE
-      else
-        raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
-      end
-      @admin.isSplitOrMergeEnabled(switch_type)
-    end
-
     def locate_region(table_name, row_key)
       locator = @connection.getRegionLocator(TableName.valueOf(table_name))
       begin

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 4144b91..0ecd3d7 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -333,8 +333,6 @@ Shell.load_command_group(
     catalogjanitor_enabled
     compact_rs
     trace
-    splitormerge_switch
-    splitormerge_enabled
   ],
   # TODO remove older hlog_roll command
   :aliases => {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb b/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb
deleted file mode 100644
index 7da7564..0000000
--- a/hbase-shell/src/main/ruby/shell/commands/splitormerge_enabled.rb
+++ /dev/null
@@ -1,41 +0,0 @@
-#!/usr/bin/env hbase-jruby
-#
-# 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.
-
-# Prints the current split or merge status
-module Shell
-  module Commands
-    # Command for check split or merge switch status
-    class SplitormergeEnabled < Command
-      def help
-        print <<-EOF
-Query the switch's state. You can set switch type, 'SPLIT' or 'MERGE'
-Examples:
-
-  hbase> splitormerge_enabled 'SPLIT'
-EOF
-      end
-
-      def command(switch_type)
-        format_simple_command do
-          formatter.row(
-            [admin.splitormerge_enabled(switch_type) ? 'true' : 'false']
-          )
-        end
-      end
-    end
-  end
-end

http://git-wip-us.apache.org/repos/asf/hbase/blob/bf4fcc30/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb b/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb
deleted file mode 100644
index f4c2858..0000000
--- a/hbase-shell/src/main/ruby/shell/commands/splitormerge_switch.rb
+++ /dev/null
@@ -1,43 +0,0 @@
-#
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-module Shell
-  module Commands
-    # Command for set switch for split and merge
-    class SplitormergeSwitch < Command
-      def help
-        print <<-EOF
-Enable/Disable one switch. You can set switch type 'SPLIT' or 'MERGE'. Returns previous split state.
-Examples:
-
-  hbase> splitormerge_switch 'SPLIT', true
-  hbase> splitormerge_switch 'SPLIT', false
-EOF
-      end
-
-      def command(switch_type, enabled)
-        format_simple_command do
-          formatter.row(
-            [admin.splitormerge_switch(switch_type, enabled) ? 'true' : 'false']
-          )
-        end
-      end
-    end
-  end
-end