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

[01/50] [abbrv] hbase git commit: HBASE-15306 Make RPC call queue length dynamically configurable

Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 9bf26f46d -> ab491d4a2


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-7912
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();
+    }
+  }
 }


[13/50] [abbrv] hbase git commit: HBASE-15264 Implement a fan out HDFS OutputStream

Posted by en...@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-7912
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);
+    }
+  }
+}


[06/50] [abbrv] hbase git commit: HBASE-15302 Reenable the other tests disabled by HBASE-14678

Posted by en...@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


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

Posted by en...@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


[34/50] [abbrv] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/99955a32/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/99955a32/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/99955a32/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/99955a32/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/99955a32/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..b4bffb4 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.SplitOrMergeTracker;
 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
+  SplitOrMergeTracker 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 SplitOrMergeTracker(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 SplitOrMergeTracker}. 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 SplitOrMergeTracker getSplitOrMergeTracker() {
+    return splitOrMergeTracker;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/99955a32/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..d6a53cd 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();
+      boolean newValue = request.getEnabled();
+      for (MasterSwitchType masterSwitchType : request.getSwitchTypesList()) {
+        Admin.MasterSwitchType switchType = convert(masterSwitchType);
+        boolean oldValue = master.isSplitOrMergeEnabled(switchType);
+        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/99955a32/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..e8a384b 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,19 @@ public class HBaseFsck extends Configured implements Closeable {
       if (shouldDisableBalancer() && oldBalancer) {
         admin.setBalancerRunning(oldBalancer, false);
       }
+
+      if (shouldDisableSplitAndMerge()) {
+        if (oldSplitAndMerge != null) {
+          if (oldSplitAndMerge[0] && oldSplitAndMerge[1]) {
+            admin.setSplitOrMergeEnabled(true, false,
+              Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE);
+          } else if (oldSplitAndMerge[0]) {
+            admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.SPLIT);
+          } else if (oldSplitAndMerge[1]) {
+            admin.setSplitOrMergeEnabled(true, false, Admin.MasterSwitchType.MERGE);
+          }
+        }
+      }
     }
 
     if (checkRegionBoundaries) {
@@ -4184,6 +4203,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 +4219,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 +4586,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/99955a32/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java
new file mode 100644
index 0000000..0d729a1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.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 SplitOrMergeTracker {
+
+  private String splitZnode;
+  private String mergeZnode;
+
+  private SwitchStateTracker splitStateTracker;
+  private SwitchStateTracker mergeStateTracker;
+
+  public SplitOrMergeTracker(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/99955a32/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/99955a32/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/99955a32/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/99955a32/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/99955a32/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


[16/50] [abbrv] hbase git commit: HBASE-15144 Procedure v2 - Web UI displaying Store state

Posted by en...@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-7912
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>
 


[48/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-protocol/src/main/protobuf/Backup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Backup.proto b/hbase-protocol/src/main/protobuf/Backup.proto
new file mode 100644
index 0000000..383b990
--- /dev/null
+++ b/hbase-protocol/src/main/protobuf/Backup.proto
@@ -0,0 +1,105 @@
+/**
+ * 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.
+ */
+
+// This file contains Backup manifest
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "BackupProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+enum BackupType {
+  FULL = 0;
+  INCREMENTAL = 1;
+}
+
+message BackupImage {
+  required string backup_id = 1;
+  required BackupType backup_type = 2;
+  required string root_dir = 3;
+  repeated TableName table_list = 4;
+  required uint64 start_ts = 5;
+  required uint64 complete_ts = 6;
+  repeated BackupImage ancestors = 7; 
+}
+
+message ServerTimestamp {
+  required string server = 1;
+  required uint64 timestamp = 2;
+}
+
+message TableServerTimestamp {
+  required TableName table = 1;
+  repeated ServerTimestamp server_timestamp = 2;
+}
+
+message BackupManifest {
+  required string version = 1;
+  required string backup_id = 2;
+  required BackupType type = 3;
+  repeated TableName table_list = 4;
+  required uint64 start_ts = 5;
+  required uint64 complete_ts = 6;
+  required int64 total_bytes = 7;
+  optional int64 log_bytes  = 8;
+  repeated TableServerTimestamp tst_map = 9;
+  repeated BackupImage dependent_backup_image = 10;
+  required bool compacted = 11; 
+}
+
+message TableBackupStatus {
+  required TableName table = 1;
+  required string target_dir = 2;
+  optional string snapshot = 3; 	
+}
+
+message BackupContext {
+  required string backup_id = 1;
+  required BackupType type = 2;
+  required string target_root_dir = 3;
+  optional BackupState state = 4;
+  optional BackupPhase phase = 5;
+  optional string failed_message = 6;
+  repeated TableBackupStatus table_backup_status = 7;
+  optional uint64  start_ts = 8;
+  optional uint64  end_ts = 9;
+  optional int64  total_bytes_copied = 10;
+  optional string hlog_target_dir = 11;
+  optional uint32 progress = 12; 
+  
+  enum BackupState {
+    WAITING = 0;
+    RUNNING = 1;
+    COMPLETE = 2;
+    FAILED = 3;
+    CANCELLED = 4;
+  }
+
+  enum BackupPhase {
+    REQUEST = 0;
+    SNAPSHOT = 1;
+    PREPARE_INCREMENTAL = 2;
+    SNAPSHOTCOPY = 3;
+    INCREMENTAL_COPY = 4;
+    STORE_MANIFEST = 5;
+  } 
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index d5f1e30..e4b296a 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -394,6 +394,11 @@
        <version>${project.version}</version>
        <optional>true</optional>
     </dependency>
+     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop-two.version}</version>
+    </dependency>
     <dependency>
       <groupId>commons-httpclient</groupId>
       <artifactId>commons-httpclient</artifactId>
@@ -407,6 +412,11 @@
       <artifactId>commons-collections</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop-two.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
new file mode 100644
index 0000000..7c8ea39
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
@@ -0,0 +1,40 @@
+/**
+ * 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.backup;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+
+public interface BackupClient {
+
+  public void setConf(Configuration conf);
+
+  /**
+   * Send backup request to server, and monitor the progress if necessary
+   * @param backupType : full or incremental
+   * @param targetRootDir : the root path specified by user
+   * @param tableList : the table list specified by user
+   * @return backupId backup id
+   * @throws IOException exception
+   */
+ public String create(BackupType backupType, List<TableName> tableList,
+      String targetRootDir) throws IOException;
+ }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
new file mode 100644
index 0000000..015c80b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
@@ -0,0 +1,119 @@
+/**
+ * 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.backup;
+
+import java.io.IOException;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.backup.impl.BackupCommands;
+import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.LogUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public class BackupDriver extends AbstractHBaseTool {
+
+  private static final Log LOG = LogFactory.getLog(BackupDriver.class);
+  private Options opt;
+  private CommandLine cmd;
+
+  protected void init() throws IOException {
+    // define supported options
+    opt = new Options();
+    opt.addOption("debug", false, "Enable debug loggings");
+
+    // disable irrelevant loggers to avoid it mess up command output
+    LogUtils.disableUselessLoggers(LOG);
+  }
+
+  private int parseAndRun(String[] args) throws IOException {
+    String cmd = null;
+    String[] remainArgs = null;
+    if (args == null || args.length == 0) {
+      BackupCommands.createCommand(getConf(),
+        BackupRestoreConstants.BackupCommand.HELP, null).execute();
+    } else {
+      cmd = args[0];
+      remainArgs = new String[args.length - 1];
+      if (args.length > 1) {
+        System.arraycopy(args, 1, remainArgs, 0, args.length - 1);
+      }
+    }
+    CommandLine cmdline = null;
+    try {
+      cmdline = new PosixParser().parse(opt, remainArgs);
+    } catch (ParseException e) {
+      LOG.error("Could not parse command", e);
+      return -1;
+    }
+
+    BackupCommand type = BackupCommand.HELP;
+    if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.CREATE;
+    } else if (BackupCommand.HELP.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.HELP;
+    } else {
+      System.out.println("Unsupported command for backup: " + cmd);
+      return -1;
+    }
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (cmdline.hasOption("debug")) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    } else {
+      backupClientLogger.setLevel(Level.INFO);
+    }
+
+    // TODO: get rid of Command altogether?
+    BackupCommands.createCommand(getConf(), type, cmdline).execute();
+    return 0;
+  }
+
+  @Override
+  protected void addOptions() {
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    this.cmd = cmd;
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    init();
+    return parseAndRun(cmd.getArgs());
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    int ret = ToolRunner.run(conf, new BackupDriver(), args);
+    System.exit(ret);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
new file mode 100644
index 0000000..6fbfe18
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
@@ -0,0 +1,95 @@
+/**
+ * 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.backup;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.impl.BackupClientImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupCopyService;
+import org.apache.hadoop.hbase.backup.impl.IncrementalRestoreService;
+import org.apache.hadoop.hbase.backup.impl.RestoreClientImpl;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyService;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreService;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.util.ReflectionUtils;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupRestoreFactory {
+
+  public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class";
+  public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class";
+  public final static String HBASE_BACKUP_CLIENT_IMPL_CLASS = "hbase.backup.client.class";
+  public final static String HBASE_RESTORE_CLIENT_IMPL_CLASS = "hbase.restore.client.class";
+
+  private BackupRestoreFactory(){
+    throw new AssertionError("Instantiating utility class...");
+  }
+  
+  /**
+   * Gets incremental restore service
+   * @param conf - configuration
+   * @return incremental backup service instance
+   */
+  public static IncrementalRestoreService getIncrementalRestoreService(Configuration conf) {
+    Class<? extends IncrementalRestoreService> cls =
+        conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreService.class,
+          IncrementalRestoreService.class);
+    return ReflectionUtils.newInstance(cls, conf);
+  }
+  
+  /**
+   * Gets backup copy service
+   * @param conf - configuration
+   * @return backup copy service
+   */
+  public static BackupCopyService getBackupCopyService(Configuration conf) {
+    Class<? extends BackupCopyService> cls =
+        conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyService.class,
+          BackupCopyService.class);
+    return ReflectionUtils.newInstance(cls, conf);
+  }
+  
+  /**
+   * Gets backup client implementation
+   * @param conf - configuration
+   * @return backup client
+   */
+  public static BackupClient getBackupClient(Configuration conf) {
+    Class<? extends BackupClient> cls =
+        conf.getClass(HBASE_BACKUP_CLIENT_IMPL_CLASS, BackupClientImpl.class,
+          BackupClient.class);
+    BackupClient client = ReflectionUtils.newInstance(cls, conf);
+    client.setConf(conf);
+    return client;
+  }
+  
+  /**
+   * Gets restore client implementation
+   * @param conf - configuration
+   * @return backup client
+   */
+  public static RestoreClient getRestoreClient(Configuration conf) {
+    Class<? extends RestoreClient> cls =
+        conf.getClass(HBASE_RESTORE_CLIENT_IMPL_CLASS, RestoreClientImpl.class,
+          RestoreClient.class);
+    RestoreClient client = ReflectionUtils.newInstance(cls, conf);
+    client.setConf(conf);
+    return client;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
new file mode 100644
index 0000000..6e5a355
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -0,0 +1,472 @@
+/**
+ *
+ * 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.backup;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * View to an on-disk Backup Image FileSytem
+ * Provides the set of methods necessary to interact with the on-disk Backup Image data.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HBackupFileSystem {
+  public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class);
+
+  private final String RESTORE_TMP_PATH = "/tmp";
+  private final String[] ignoreDirs = { "recovered.edits" };
+
+  private final Configuration conf;
+  private final FileSystem fs;
+  private final Path backupRootPath;
+  private final Path restoreTmpPath;
+  private final String backupId;
+
+  /**
+   * Create a view to the on-disk Backup Image.
+   * @param conf  to use
+   * @param backupPath  to where the backup Image stored
+   * @param backupId represent backup Image
+   */
+  public HBackupFileSystem(final Configuration conf, final Path backupRootPath, final String backupId)
+      throws IOException {
+    this.conf = conf;
+    this.fs = backupRootPath.getFileSystem(conf);
+    this.backupRootPath = backupRootPath;
+    this.backupId = backupId; // the backup ID for the lead backup Image
+    this.restoreTmpPath = new Path(conf.get("hbase.fs.tmp.dir") != null?
+          conf.get("hbase.fs.tmp.dir"): RESTORE_TMP_PATH,
+        "restore");
+  }
+
+  public Path getBackupRootPath() {
+    return backupRootPath;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  /**
+   * @param tableName is the table backed up
+   * @return {@link HTableDescriptor} saved in backup image of the table
+   */
+  public HTableDescriptor getTableDesc(TableName tableName)
+      throws FileNotFoundException, IOException {
+    Path tableInfoPath = this.getTableInfoPath(tableName);
+    SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
+    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
+    HTableDescriptor tableDescriptor = manifest.getTableDescriptor();
+    if (!tableDescriptor.getNameAsString().equals(tableName)) {
+      LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
+          + tableInfoPath.toString());
+      LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString());
+    }
+    return tableDescriptor;
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/default/t1_dn/backup_1396650096738"
+   * @param backupRootDir backup root directory
+   * @param backupId  backup id
+   * @param table table name
+   * @return backupPath String for the particular table
+   */
+  public static String getTableBackupDir(String backupRootDir, String backupId,
+      TableName tableName) {
+    return backupRootDir + Path.SEPARATOR + tableName.getNamespaceAsString() + Path.SEPARATOR
+        + tableName.getQualifierAsString() + Path.SEPARATOR + backupId;
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/default/t1_dn/backup_1396650096738"
+   * @param backupRootPath backup root path
+   * @param tableName table name
+   * @param backupId backup Id
+   * @return backupPath for the particular table
+   */
+  public static Path getTableBackupPath(Path backupRootPath, TableName tableName, String backupId) {
+    return new Path(backupRootPath, tableName.getNamespaceAsString() + Path.SEPARATOR
+      + tableName.getQualifierAsString() + Path.SEPARATOR + backupId);
+  }
+
+  /**
+   * return value represent path for:
+   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/.hbase-snapshot"
+   * @param backupRootPath backup root path
+   * @param tableName table name
+   * @param backupId backup Id
+   * @return path for snapshot
+   */
+  public static Path getTableSnapshotPath(Path backupRootPath, TableName tableName,
+      String backupId) {
+    return new Path(getTableBackupPath(backupRootPath, tableName, backupId),
+      HConstants.SNAPSHOT_DIR_NAME);
+  }
+
+  /**
+   * return value represent path for:
+   * "..../default/t1_dn/backup_1396650096738/.hbase-snapshot/snapshot_1396650097621_default_t1_dn"
+   * this path contains .snapshotinfo, .tabledesc (0.96 and 0.98) this path contains .snapshotinfo,
+   * .data.manifest (trunk)
+   * @param tableName table name
+   * @return path to table info
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+  public Path getTableInfoPath(TableName tableName)
+      throws FileNotFoundException, IOException {
+    Path tableSnapShotPath = getTableSnapshotPath(backupRootPath, tableName, backupId);
+    Path tableInfoPath = null;
+
+    // can't build the path directly as the timestamp values are different
+    FileStatus[] snapshots = fs.listStatus(tableSnapShotPath);
+    for (FileStatus snapshot : snapshots) {
+      tableInfoPath = snapshot.getPath();
+      // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest";
+      if (tableInfoPath.getName().endsWith("data.manifest")) {
+        break;
+      }
+    }
+    return tableInfoPath;
+  }
+
+  /**
+   * return value represent path for:
+   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
+   * @param tabelName table name
+   * @return path to table archive
+   * @throws IOException exception
+   */
+  public Path getTableArchivePath(TableName tableName)
+      throws IOException {
+    Path baseDir = new Path(getTableBackupPath(backupRootPath, tableName, backupId),
+      HConstants.HFILE_ARCHIVE_DIRECTORY);
+    Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR);
+    Path archivePath = new Path(dataDir, tableName.getNamespaceAsString());
+    Path tableArchivePath =
+        new Path(archivePath, tableName.getQualifierAsString());
+    if (!fs.exists(tableArchivePath) || !fs.getFileStatus(tableArchivePath).isDirectory()) {
+      LOG.debug("Folder tableArchivePath: " + tableArchivePath.toString() + " does not exists");
+      tableArchivePath = null; // empty table has no archive
+    }
+    return tableArchivePath;
+  }
+
+  /**
+   * Given the backup root dir and the backup id, return the log file location for an incremental
+   * backup.
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738"
+   */
+  public static String getLogBackupDir(String backupRootDir, String backupId) {
+    return backupRootDir + Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME + Path.SEPARATOR
+        + backupId;
+  }
+
+  public static Path getLogBackupPath(String backupRootDir, String backupId) {
+    return new Path(getLogBackupDir(backupRootDir, backupId));
+  }
+
+  private static Path getManifestPath(TableName tableName, Configuration conf,
+      Path backupRootPath, String backupId) throws IOException {
+    Path manifestPath = new Path(getTableBackupPath(backupRootPath, tableName, backupId),
+      BackupManifest.MANIFEST_FILE_NAME);
+    FileSystem fs = backupRootPath.getFileSystem(conf);
+    if (!fs.exists(manifestPath)) {
+      // check log dir for incremental backup case
+      manifestPath =
+          new Path(getLogBackupDir(backupRootPath.toString(), backupId) + Path.SEPARATOR
+            + BackupManifest.MANIFEST_FILE_NAME);
+      if (!fs.exists(manifestPath)) {
+        String errorMsg =
+            "Could not find backup manifest for " + backupId + " in " + backupRootPath.toString();
+        throw new IOException(errorMsg);
+      }
+    }
+    return manifestPath;
+  }
+
+  public static BackupManifest getManifest(TableName tableName, Configuration conf,
+      Path backupRootPath, String backupId) throws IOException {
+    BackupManifest manifest = new BackupManifest(conf,
+      getManifestPath(tableName, conf, backupRootPath, backupId));
+    return manifest;
+  }
+
+  /**
+   * Gets region list
+   * @param tableName table name
+   * @return RegionList region list
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+
+  public ArrayList<Path> getRegionList(TableName tableName)
+      throws FileNotFoundException, IOException {
+    Path tableArchivePath = this.getTableArchivePath(tableName);
+    ArrayList<Path> regionDirList = new ArrayList<Path>();
+    FileStatus[] children = fs.listStatus(tableArchivePath);
+    for (FileStatus childStatus : children) {
+      // here child refer to each region(Name)
+      Path child = childStatus.getPath();
+      regionDirList.add(child);
+    }
+    return regionDirList;
+  }
+
+  /**
+   * Gets region list
+   * @param tableArchivePath table archive path
+   * @return RegionList region list
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+  public ArrayList<Path> getRegionList(Path tableArchivePath) throws FileNotFoundException,
+  IOException {
+    ArrayList<Path> regionDirList = new ArrayList<Path>();
+    FileStatus[] children = fs.listStatus(tableArchivePath);
+    for (FileStatus childStatus : children) {
+      // here child refer to each region(Name)
+      Path child = childStatus.getPath();
+      regionDirList.add(child);
+    }
+    return regionDirList;
+  }
+
+  /**
+   * Counts the number of files in all subdirectories of an HBase tables, i.e. HFiles. And finds the
+   * maximum number of files in one HBase table.
+   * @param tableArchivePath archive path
+   * @return the maximum number of files found in 1 HBase table
+   * @throws IOException exception
+   */
+  public int getMaxNumberOfFilesInSubDir(Path tableArchivePath) throws IOException {
+    int result = 1;
+    ArrayList<Path> regionPathList = this.getRegionList(tableArchivePath);
+    // tableArchivePath = this.getTableArchivePath(tableName);
+
+    if (regionPathList == null || regionPathList.size() == 0) {
+      throw new IllegalStateException("Cannot restore hbase table because directory '"
+          + tableArchivePath + "' is not a directory.");
+    }
+
+    for (Path regionPath : regionPathList) {
+      result = Math.max(result, getNumberOfFilesInDir(regionPath));
+    }
+    return result;
+  }
+
+  /**
+   * Counts the number of files in all subdirectories of an HBase table, i.e. HFiles.
+   * @param regionPath Path to an HBase table directory
+   * @return the number of files all directories
+   * @throws IOException exception
+   */
+  public int getNumberOfFilesInDir(Path regionPath) throws IOException {
+    int result = 0;
+
+    if (!fs.exists(regionPath) || !fs.getFileStatus(regionPath).isDirectory()) {
+      throw new IllegalStateException("Cannot restore hbase table because directory '"
+          + regionPath.toString() + "' is not a directory.");
+    }
+
+    FileStatus[] tableDirContent = fs.listStatus(regionPath);
+    for (FileStatus subDirStatus : tableDirContent) {
+      FileStatus[] colFamilies = fs.listStatus(subDirStatus.getPath());
+      for (FileStatus colFamilyStatus : colFamilies) {
+        FileStatus[] colFamilyContent = fs.listStatus(colFamilyStatus.getPath());
+        result += colFamilyContent.length;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Duplicate the backup image if it's on local cluster
+   * @see HStore#bulkLoadHFile(String, long)
+   * @see HRegionFileSystem#bulkLoadStoreFile(String familyName, Path srcPath, long seqNum)
+   * @param tableArchivePath archive path
+   * @return the new tableArchivePath
+   * @throws IOException exception
+   */
+  public Path checkLocalAndBackup(Path tableArchivePath) throws IOException {
+    // Move the file if it's on local cluster
+    boolean isCopyNeeded = false;
+
+    FileSystem srcFs = tableArchivePath.getFileSystem(conf);
+    FileSystem desFs = FileSystem.get(conf);
+    if (tableArchivePath.getName().startsWith("/")) {
+      isCopyNeeded = true;
+    } else {
+      // This should match what is done in @see HRegionFileSystem#bulkLoadStoreFile(String, Path,
+      // long)
+      if (srcFs.getUri().equals(desFs.getUri())) {
+        LOG.debug("cluster hold the backup image: " + srcFs.getUri() + "; local cluster node: "
+            + desFs.getUri());
+        isCopyNeeded = true;
+      }
+    }
+    if (isCopyNeeded) {
+      LOG.debug("File " + tableArchivePath + " on local cluster, back it up before restore");
+      if (desFs.exists(restoreTmpPath)) {
+        try {
+          desFs.delete(restoreTmpPath, true);
+        } catch (IOException e) {
+          LOG.debug("Failed to delete path: " + restoreTmpPath
+            + ", need to check whether restore target DFS cluster is healthy");
+        }
+      }
+      FileUtil.copy(srcFs, tableArchivePath, desFs, restoreTmpPath, false, conf);
+      LOG.debug("Copied to temporary path on local cluster: " + restoreTmpPath);
+      tableArchivePath = restoreTmpPath;
+    }
+    return tableArchivePath;
+  }
+
+  /**
+   * Calculate region boundaries and add all the column families to the table descriptor
+   * @param regionDirList region dir list
+   * @return a set of keys to store the boundaries
+   */
+  public byte[][] generateBoundaryKeys(ArrayList<Path> regionDirList)
+      throws FileNotFoundException, IOException {
+    TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    // Build a set of keys to store the boundaries
+    byte[][] keys = null;
+    // calculate region boundaries and add all the column families to the table descriptor
+    for (Path regionDir : regionDirList) {
+      LOG.debug("Parsing region dir: " + regionDir);
+      Path hfofDir = regionDir;
+
+      if (!fs.exists(hfofDir)) {
+        LOG.warn("HFileOutputFormat dir " + hfofDir + " not found");
+      }
+
+      FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
+      if (familyDirStatuses == null) {
+        throw new IOException("No families found in " + hfofDir);
+      }
+
+      for (FileStatus stat : familyDirStatuses) {
+        if (!stat.isDirectory()) {
+          LOG.warn("Skipping non-directory " + stat.getPath());
+          continue;
+        }
+        boolean isIgnore = false;
+        String pathName = stat.getPath().getName();
+        for (String ignore : ignoreDirs) {
+          if (pathName.contains(ignore)) {
+            LOG.warn("Skipping non-family directory" + pathName);
+            isIgnore = true;
+            break;
+          }
+        }
+        if (isIgnore) {
+          continue;
+        }
+        Path familyDir = stat.getPath();
+        LOG.debug("Parsing family dir [" + familyDir.toString() + " in region [" + regionDir + "]");
+        // Skip _logs, etc
+        if (familyDir.getName().startsWith("_") || familyDir.getName().startsWith(".")) {
+          continue;
+        }
+
+        // start to parse hfile inside one family dir
+        Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
+        for (Path hfile : hfiles) {
+          if (hfile.getName().startsWith("_") || hfile.getName().startsWith(".")
+              || StoreFileInfo.isReference(hfile.getName())
+              || HFileLink.isHFileLink(hfile.getName())) {
+            continue;
+          }
+          HFile.Reader reader = HFile.createReader(fs, hfile, new CacheConfig(conf), conf);
+          final byte[] first, last;
+          try {
+            reader.loadFileInfo();
+            first = reader.getFirstRowKey();
+            last = reader.getLastRowKey();
+            LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first="
+                + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
+
+            // To eventually infer start key-end key boundaries
+            Integer value = map.containsKey(first) ? (Integer) map.get(first) : 0;
+            map.put(first, value + 1);
+            value = map.containsKey(last) ? (Integer) map.get(last) : 0;
+            map.put(last, value - 1);
+          } finally {
+            reader.close();
+          }
+        }
+      }
+    }
+    keys = LoadIncrementalHFiles.inferBoundaries(map);
+    return keys;
+  }
+
+  /**
+   * Check whether the backup image path and there is manifest file in the path.
+   * @param backupManifestMap If all the manifests are found, then they are put into this map
+   * @param tableArray the tables involved
+   * @throws IOException exception
+   */
+  public static void checkImageManifestExist(HashMap<TableName, BackupManifest> backupManifestMap,
+      TableName[] tableArray, Configuration conf,
+      Path backupRootPath, String backupId) throws IOException {
+    for (TableName tableName : tableArray) {
+      BackupManifest manifest = getManifest(tableName, conf, backupRootPath, backupId);
+      backupManifestMap.put(tableName, manifest);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
new file mode 100644
index 0000000..a3aaa98
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
@@ -0,0 +1,46 @@
+/**
+ * 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.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+
+public interface RestoreClient {
+
+  public void setConf(Configuration conf);
+
+  /**
+   * Restore operation.
+   * @param backupRootDir The root dir for backup image
+   * @param backupId The backup id for image to be restored
+   * @param check True if only do dependency check
+   * @param autoRestore True if automatically restore following the dependency
+   * @param sTableArray The array of tables to be restored
+   * @param tTableArray The array of mapping tables to restore to
+   * @param isOverwrite True then do restore overwrite if target table exists, otherwise fail the
+   *          request if target table exists
+   * @return True if only do dependency check
+   * @throws IOException if any failure during restore
+   */
+  public  boolean restore(
+      String backupRootDir,
+      String backupId, boolean check, boolean autoRestore, TableName[] sTableArray,
+      TableName[] tTableArray, boolean isOverwrite) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
new file mode 100644
index 0000000..541882a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -0,0 +1,174 @@
+/**
+ * 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.backup;
+
+import java.io.IOException;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupUtil;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.LogUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public class RestoreDriver extends AbstractHBaseTool {
+
+  private static final Log LOG = LogFactory.getLog(BackupDriver.class);
+  private Options opt;
+  private CommandLine cmd;
+
+  private static final String OPTION_OVERWRITE = "overwrite";
+  private static final String OPTION_CHECK = "check";
+  private static final String OPTION_AUTOMATIC = "automatic";
+
+  private static final String USAGE =
+      "Usage: hbase restore <backup_root_path> <backup_id> <tables> [tableMapping] \n"
+          + "       [-overwrite] [-check] [-automatic]\n"
+          + " backup_root_path  The parent location where the backup images are stored\n"
+          + " backup_id         The id identifying the backup image\n"
+          + " table(s)          Table(s) from the backup image to be restored.\n"
+          + "                   Tables are separated by comma.\n"
+          + " Options:\n"
+          + "   tableMapping    A comma separated list of target tables.\n"
+          + "                   If specified, each table in <tables> must have a mapping.\n"
+          + "   -overwrite      With this option, restore overwrites to the existing table "
+          + "if there's any in\n"
+          + "                   restore target. The existing table must be online before restore.\n"
+          + "   -check          With this option, restore sequence and dependencies are checked\n"
+          + "                   and verified without executing the restore\n"
+          + "   -automatic      With this option, all the dependencies are automatically restored\n"
+          + "                   together with this backup image following the correct order.\n"
+          + "                   The restore dependencies can be checked by using \"-check\" "
+          + "option,\n"
+          + "                   or using \"hbase backup describe\" command. Without this option, "
+          + "only\n" + "                   this backup image is restored\n";
+
+  protected void init() throws IOException {
+    // define supported options
+    opt = new Options();
+    opt.addOption(OPTION_OVERWRITE, false,
+        "Overwrite the data if any of the restore target tables exists");
+    opt.addOption(OPTION_CHECK, false, "Check restore sequence and dependencies");
+    opt.addOption(OPTION_AUTOMATIC, false, "Restore all dependencies");
+    opt.addOption("debug", false, "Enable debug logging");
+
+    // disable irrelevant loggers to avoid it mess up command output
+    LogUtils.disableUselessLoggers(LOG);
+  }
+
+  private int parseAndRun(String[] args) {
+    CommandLine cmd = null;
+    try {
+      cmd = new PosixParser().parse(opt, args);
+    } catch (ParseException e) {
+      LOG.error("Could not parse command", e);
+      return -1;
+    }
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (cmd.hasOption("debug")) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    }
+
+    // whether to overwrite to existing table if any, false by default
+    boolean isOverwrite = cmd.hasOption(OPTION_OVERWRITE);
+    if (isOverwrite) {
+      LOG.debug("Found -overwrite option in restore command, "
+          + "will overwrite to existing table if any in the restore target");
+    }
+
+    // whether to only check the dependencies, false by default
+    boolean check = cmd.hasOption(OPTION_CHECK);
+    if (check) {
+      LOG.debug("Found -check option in restore command, "
+          + "will check and verify the dependencies");
+    }
+
+    // whether to restore all dependencies, false by default
+    boolean autoRestore = cmd.hasOption(OPTION_AUTOMATIC);
+    if (autoRestore) {
+      LOG.debug("Found -automatic option in restore command, "
+          + "will automatically retore all the dependencies");
+    }
+
+    // parse main restore command options
+    String[] remainArgs = cmd.getArgs();
+    if (remainArgs.length < 3) {
+      System.out.println("ERROR: missing arguments");
+      System.out.println(USAGE);
+      return -1;
+    }
+
+    String backupRootDir = remainArgs[0];
+    String backupId = remainArgs[1];
+    String tables = remainArgs[2];
+
+    String tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null;
+
+    TableName[] sTableArray = BackupUtil.parseTableNames(tables);
+    TableName[] tTableArray = BackupUtil.parseTableNames(tableMapping);
+
+    if (sTableArray != null && tTableArray != null && (sTableArray.length != tTableArray.length)) {
+      System.err.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
+      System.out.println(USAGE);
+      return -1;
+    }
+
+    try {
+      RestoreClient client = BackupRestoreFactory.getRestoreClient(conf);
+      client.restore(backupRootDir, backupId, check, autoRestore, sTableArray,
+        tTableArray, isOverwrite);
+    } catch (IOException e) {
+      System.err.println("ERROR: " + e.getMessage());
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override
+  protected void addOptions() {
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    this.cmd = cmd;
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    init();
+    return parseAndRun(cmd.getArgs());
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    int ret = ToolRunner.run(conf, new BackupDriver(), args);
+    System.exit(ret);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
new file mode 100644
index 0000000..5b8a151
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
@@ -0,0 +1,183 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+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.DoNotRetryIOException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupClient;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Backup HBase tables locally or on a remote cluster Serve as client entry point for the following
+ * features: - Full Backup provide local and remote back/restore for a list of tables - Incremental
+ * backup to build on top of full backup as daily/weekly backup - Convert incremental backup WAL
+ * files into hfiles - Merge several backup images into one(like merge weekly into monthly) - Add
+ * and remove table to and from Backup image - Cancel a backup process - Describe information of
+ * a backup image
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupClientImpl implements BackupClient {
+  private static final Log LOG = LogFactory.getLog(BackupClientImpl.class);
+  private Configuration conf;
+  private BackupManager backupManager;
+
+  public BackupClientImpl() {
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Prepare and submit Backup request
+   * @param backupId : backup_timestame (something like backup_1398729212626)
+   * @param backupType : full or incremental
+   * @param tableList : tables to be backuped
+   * @param targetRootDir : specified by user
+   * @throws IOException exception
+   */
+  protected void requestBackup(String backupId, BackupType backupType, List<TableName> tableList,
+      String targetRootDir) throws IOException {
+
+    BackupContext backupContext = null;
+
+    HBaseAdmin hbadmin = null;
+    Connection conn = null;
+    try {
+      backupManager = new BackupManager(conf);
+      if (backupType == BackupType.INCREMENTAL) {
+        Set<TableName> incrTableSet = backupManager.getIncrementalBackupTableSet();
+        if (incrTableSet.isEmpty()) {
+          LOG.warn("Incremental backup table set contains no table.\n"
+              + "Use 'backup create full' or 'backup stop' to \n "
+              + "change the tables covered by incremental backup.");
+          throw new DoNotRetryIOException("No table covered by incremental backup.");
+        }
+
+        LOG.info("Incremental backup for the following table set: " + incrTableSet);
+        tableList = Lists.newArrayList(incrTableSet);
+      }
+
+      // check whether table exists first before starting real request
+      if (tableList != null) {
+        ArrayList<TableName> nonExistingTableList = null;
+        conn = ConnectionFactory.createConnection(conf);
+        hbadmin = (HBaseAdmin) conn.getAdmin();
+        for (TableName tableName : tableList) {
+          if (!hbadmin.tableExists(tableName)) {
+            if (nonExistingTableList == null) {
+              nonExistingTableList = new ArrayList<>();
+            }
+            nonExistingTableList.add(tableName);
+          }
+        }
+        if (nonExistingTableList != null) {
+          if (backupType == BackupType.INCREMENTAL ) {
+            LOG.warn("Incremental backup table set contains non-exising table: "
+                + nonExistingTableList);
+          } else {
+            // Throw exception only in full mode - we try to backup non-existing table
+            throw new DoNotRetryIOException("Non-existing tables found in the table list: "
+                + nonExistingTableList);
+          }
+        }
+      }
+
+      // if any target table backup dir already exist, then no backup action taken
+      if (tableList != null) {
+        for (TableName table : tableList) {
+          String targetTableBackupDir =
+              HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+          Path targetTableBackupDirPath = new Path(targetTableBackupDir);
+          FileSystem outputFs = FileSystem.get(targetTableBackupDirPath.toUri(), conf);
+          if (outputFs.exists(targetTableBackupDirPath)) {
+            throw new DoNotRetryIOException("Target backup directory " + targetTableBackupDir
+              + " exists already.");
+          }
+        }
+      }
+      backupContext =
+          backupManager.createBackupContext(backupId, backupType, tableList, targetRootDir);
+      backupManager.initialize();
+      backupManager.dispatchRequest(backupContext);
+    } catch (BackupException e) {
+      // suppress the backup exception wrapped within #initialize or #dispatchRequest, backup
+      // exception has already been handled normally
+      LOG.error("Backup Exception ", e);
+    } finally {
+      if (hbadmin != null) {
+        hbadmin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+  }
+
+  @Override
+  public String create(BackupType backupType, List<TableName> tableList, String backupRootPath)
+      throws IOException {
+
+    String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
+    BackupUtility.checkTargetDir(backupRootPath, conf);
+
+    // table list specified for backup, trigger backup on specified tables
+    try {
+      requestBackup(backupId, backupType, tableList, backupRootPath);
+    } catch (RuntimeException e) {
+      String errMsg = e.getMessage();
+      if (errMsg != null
+          && (errMsg.startsWith("Non-existing tables found") || errMsg
+              .startsWith("Snapshot is not found"))) {
+        LOG.error(errMsg + ", please check your command");
+        throw e;
+      } else {
+        throw e;
+      }
+    } finally{
+      if(backupManager != null) {
+        backupManager.close();
+      }
+    }
+    return backupId;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
new file mode 100644
index 0000000..56e26fa
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -0,0 +1,158 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.backup.BackupClient;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+import com.google.common.collect.Lists;
+
+/**
+ * General backup commands, options and usage messages
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupCommands {
+
+  private static final String USAGE = "Usage: hbase backup COMMAND\n"
+      + "where COMMAND is one of:\n" + "  create     create a new backup image\n"
+      + "Enter \'help COMMAND\' to see help message for each command\n";
+
+  private static final String CREATE_CMD_USAGE =
+      "Usage: hbase backup create <type> <backup_root_path> [tables] [-convert] "
+          + "\n" + " type          \"full\" to create a full backup image;\n"
+          + "               \"incremental\" to create an incremental backup image\n"
+          + " backup_root_path   The full root path to store the backup image,\n"
+          + "                    the prefix can be hdfs, webhdfs, gpfs, etc\n" + " Options:\n"
+          + "   tables      If no tables (\"\") are specified, all tables are backed up. "
+          + "Otherwise it is a\n" + "               comma separated list of tables.\n"
+          + "   -convert    For an incremental backup, convert WAL files to HFiles\n";
+
+  public static abstract class Command extends Configured {
+    Command(Configuration conf) {
+      super(conf);
+    }
+    public abstract void execute() throws IOException;
+  }
+
+  private BackupCommands() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) {
+    Command cmd = null;
+    switch (type) {
+      case CREATE:
+        cmd = new CreateCommand(conf, cmdline);
+        break;
+      case HELP:
+      default:
+        cmd = new HelpCommand(conf, cmdline);
+        break;
+    }
+    return cmd;
+  }
+
+  private static class CreateCommand extends Command {
+    CommandLine cmdline;
+
+    CreateCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null) {
+        System.out.println("ERROR: missing arguments");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length < 2 || args.length > 3) {
+        System.out.println("ERROR: wrong number of arguments");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      if (!BackupType.FULL.toString().equalsIgnoreCase(args[0])
+          && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[0])) {
+        System.out.println("ERROR: invalid backup type");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      String tables = (args.length == 3) ? args[2] : null;
+
+      try {
+        BackupClient client = BackupRestoreFactory.getBackupClient(getConf());
+        client.create(BackupType.valueOf(args[0].toUpperCase()),
+          Lists.newArrayList(BackupUtil.parseTableNames(tables)), args[1]);
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+  }
+
+  private static class HelpCommand extends Command {
+    CommandLine cmdline;
+
+    HelpCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null) {
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      String[] args = cmdline.getArgs();
+      if (args == null || args.length == 0) {
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      if (args.length != 1) {
+        System.out.println("Only support check help message of a single command type");
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      String type = args[0];
+
+      if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) {
+        System.out.println(CREATE_CMD_USAGE);
+      } // other commands will be supported in future jira
+      System.exit(0);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java
new file mode 100644
index 0000000..1be0c3b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupContext.java
@@ -0,0 +1,382 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupContext.Builder;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.TableBackupStatus;
+
+/**
+ * An object to encapsulate the information for each backup request
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupContext {
+
+  public Map<TableName, BackupStatus> getBackupStatusMap() {
+    return backupStatusMap;
+  }
+
+  public void setBackupStatusMap(Map<TableName, BackupStatus> backupStatusMap) {
+    this.backupStatusMap = backupStatusMap;
+  }
+
+  public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() {
+    return tableSetTimestampMap;
+  }
+
+  public void setTableSetTimestampMap(
+      HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap) {
+    this.tableSetTimestampMap = tableSetTimestampMap;
+  }
+
+  public String getHlogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  public void setTargetRootDir(String targetRootDir) {
+    this.targetRootDir = targetRootDir;
+  }
+
+  public void setTotalBytesCopied(long totalBytesCopied) {
+    this.totalBytesCopied = totalBytesCopied;
+  }
+
+  public void setCancelled(boolean cancelled) {
+    this.state = BackupState.CANCELLED;;
+  }
+
+  // backup id: a timestamp when we request the backup
+  private String backupId;
+
+  // backup type, full or incremental
+  private BackupType type;
+
+  // target root directory for storing the backup files
+  private String targetRootDir;
+
+  // overall backup state
+  private BackupHandler.BackupState state;
+
+  // overall backup phase
+  private BackupHandler.BackupPhase phase;
+
+  // overall backup failure message
+  private String failedMsg;
+
+  // backup status map for all tables
+  private Map<TableName, BackupStatus> backupStatusMap;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual end timestamp of the backup process, could be fail or complete
+  private long endTs;
+
+  // the total bytes of incremental logs copied
+  private long totalBytesCopied;
+
+  // for incremental backup, the location of the backed-up hlogs
+  private String hlogTargetDir = null;
+
+  // incremental backup file list
+  transient private List<String> incrBackupFileList;
+
+  // new region server log timestamps for table set after distributed log roll
+  // key - table name, value - map of RegionServer hostname -> last log rolled timestamp
+  transient private HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap;
+
+  // backup progress in %% (0-100)
+
+  private int progress;
+
+  public BackupContext() {
+  }
+
+  public BackupContext(String backupId, BackupType type, TableName[] tables, String targetRootDir) {
+    backupStatusMap = new HashMap<TableName, BackupStatus>();
+
+    this.backupId = backupId;
+    this.type = type;
+    this.targetRootDir = targetRootDir;
+
+    this.addTables(tables);
+
+    if (type == BackupType.INCREMENTAL) {
+      setHlogTargetDir(HBackupFileSystem.getLogBackupDir(targetRootDir, backupId));
+    }
+
+    this.startTs = 0;
+    this.endTs = 0;
+  }
+
+  /**
+   * Set progress string
+   * @param msg progress message
+   */
+
+  public void setProgress(int p) {
+    this.progress = p;
+  }
+
+  /**
+   * Get current progress
+   */
+  public int getProgress() {
+    return progress;
+  }
+
+
+  /**
+   * Has been marked as cancelled or not.
+   * @return True if marked as cancelled
+   */
+  public boolean isCancelled() {
+    return this.state == BackupState.CANCELLED;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public void setBackupId(String backupId) {
+    this.backupId = backupId;
+  }
+
+  public BackupStatus getBackupStatus(TableName table) {
+    return this.backupStatusMap.get(table);
+  }
+
+  public String getFailedMsg() {
+    return failedMsg;
+  }
+
+  public void setFailedMsg(String failedMsg) {
+    this.failedMsg = failedMsg;
+  }
+
+  public long getStartTs() {
+    return startTs;
+  }
+
+  public void setStartTs(long startTs) {
+    this.startTs = startTs;
+  }
+
+  public long getEndTs() {
+    return endTs;
+  }
+
+  public void setEndTs(long endTs) {
+    this.endTs = endTs;
+  }
+
+  public long getTotalBytesCopied() {
+    return totalBytesCopied;
+  }
+
+  public BackupHandler.BackupState getState() {
+    return state;
+  }
+
+  public void setState(BackupHandler.BackupState flag) {
+    this.state = flag;
+  }
+
+  public BackupHandler.BackupPhase getPhase() {
+    return phase;
+  }
+
+  public void setPhase(BackupHandler.BackupPhase phase) {
+    this.phase = phase;
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setSnapshotName(TableName table, String snapshotName) {
+    this.backupStatusMap.get(table).setSnapshotName(snapshotName);
+  }
+
+  public String getSnapshotName(TableName table) {
+    return this.backupStatusMap.get(table).getSnapshotName();
+  }
+
+  public List<String> getSnapshotNames() {
+    List<String> snapshotNames = new ArrayList<String>();
+    for (BackupStatus backupStatus : this.backupStatusMap.values()) {
+      snapshotNames.add(backupStatus.getSnapshotName());
+    }
+    return snapshotNames;
+  }
+
+  public Set<TableName> getTables() {
+    return this.backupStatusMap.keySet();
+  }
+
+  public List<TableName> getTableNames() {
+    return new ArrayList<TableName>(backupStatusMap.keySet());
+  }
+
+  public void addTables(TableName[] tables) {
+    for (TableName table : tables) {
+      BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
+      this.backupStatusMap.put(table, backupStatus);
+    }
+  }
+
+  public String getTargetRootDir() {
+    return targetRootDir;
+  }
+
+  public void setHlogTargetDir(String hlogTagetDir) {
+    this.hlogTargetDir = hlogTagetDir;
+  }
+
+  public String getHLogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public List<String> getIncrBackupFileList() {
+    return incrBackupFileList;
+  }
+
+  public List<String> setIncrBackupFileList(List<String> incrBackupFileList) {
+    this.incrBackupFileList = incrBackupFileList;
+    return this.incrBackupFileList;
+  }
+
+  /**
+   * Set the new region server log timestamps after distributed log roll
+   * @param newTableSetTimestampMap table timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<TableName,
+      HashMap<String, Long>> newTableSetTimestampMap) {
+    this.tableSetTimestampMap = newTableSetTimestampMap;
+  }
+
+  /**
+   * Get new region server log timestamps after distributed log roll
+   * @return new region server log timestamps
+   */
+  public HashMap<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    return this.tableSetTimestampMap;
+  }
+
+  public TableName getTableBySnapshot(String snapshotName) {
+    for (Entry<TableName, BackupStatus> entry : this.backupStatusMap.entrySet()) {
+      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public byte[] toByteArray() throws IOException {
+    BackupProtos.BackupContext.Builder builder =
+        BackupProtos.BackupContext.newBuilder();
+    builder.setBackupId(getBackupId());
+    setBackupStatusMap(builder);
+    builder.setEndTs(getEndTs());
+    if(getFailedMsg() != null){
+      builder.setFailedMessage(getFailedMsg());
+    }
+    if(getState() != null){
+      builder.setState(BackupProtos.BackupContext.BackupState.valueOf(getState().name()));
+    }
+    if(getPhase() != null){
+      builder.setPhase(BackupProtos.BackupContext.BackupPhase.valueOf(getPhase().name()));
+    }
+    if(getHLogTargetDir() != null){
+      builder.setHlogTargetDir(getHLogTargetDir());
+    }
+
+    builder.setProgress(getProgress());
+    builder.setStartTs(getStartTs());
+    builder.setTargetRootDir(getTargetRootDir());
+    builder.setTotalBytesCopied(getTotalBytesCopied());
+    builder.setType(BackupProtos.BackupType.valueOf(getType().name()));
+    byte[] data = builder.build().toByteArray();
+    return data;
+  }
+
+  private void setBackupStatusMap(Builder builder) {
+    for (Entry<TableName, BackupStatus> entry: backupStatusMap.entrySet()) {
+      builder.addTableBackupStatus(entry.getValue().toProto());
+    }
+  }
+
+  public static BackupContext fromByteArray(byte[] data) throws IOException {
+
+    BackupContext context = new BackupContext();
+    BackupProtos.BackupContext proto = BackupProtos.BackupContext.parseFrom(data);
+    context.setBackupId(proto.getBackupId());
+    context.setBackupStatusMap(toMap(proto.getTableBackupStatusList()));
+    context.setEndTs(proto.getEndTs());
+    if(proto.hasFailedMessage()) {
+      context.setFailedMsg(proto.getFailedMessage());
+    }
+    if(proto.hasState()) {
+      context.setState(BackupHandler.BackupState.valueOf(proto.getState().name()));
+    }
+    if(proto.hasHlogTargetDir()) {
+      context.setHlogTargetDir(proto.getHlogTargetDir());
+    }
+    if(proto.hasPhase()) {
+      context.setPhase(BackupHandler.BackupPhase.valueOf(proto.getPhase().name()));
+    }
+    if(proto.hasProgress()) {
+      context.setProgress(proto.getProgress());
+    }
+    context.setStartTs(proto.getStartTs());
+    context.setTargetRootDir(proto.getTargetRootDir());
+    context.setTotalBytesCopied(proto.getTotalBytesCopied());
+    context.setType(BackupType.valueOf(proto.getType().name()));
+    return context;
+  }
+
+  private static Map<TableName, BackupStatus> toMap(List<TableBackupStatus> list) {
+    HashMap<TableName, BackupStatus> map = new HashMap<>();
+    for (TableBackupStatus tbs : list){
+      map.put(ProtobufUtil.toTableName(tbs.getTable()), BackupStatus.convert(tbs));
+    }
+    return map;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java
new file mode 100644
index 0000000..1e8da63
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.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.backup.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BackupCopyService extends Configurable {
+  static enum Type {
+    FULL, INCREMENTAL
+  }
+
+  public int copy(BackupContext backupContext, BackupManager backupManager, Configuration conf,
+      BackupCopyService.Type copyType, String[] options) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
new file mode 100644
index 0000000..af70cc8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
@@ -0,0 +1,85 @@
+/**
+ * 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.backup.impl;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Backup exception
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupException extends HBaseIOException {
+  private BackupContext description;
+
+  /**
+   * Some exception happened for a backup and don't even know the backup that it was about
+   * @param msg Full description of the failure
+   */
+  public BackupException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * Some exception happened for a backup with a cause
+   * @param cause the cause
+   */
+  public BackupException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * Exception for the given backup that has no previous root cause
+   * @param msg reason why the backup failed
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, BackupContext desc) {
+    super(msg);
+    this.description = desc;
+  }
+
+  /**
+   * Exception for the given backup due to another exception
+   * @param msg reason why the backup failed
+   * @param cause root cause of the failure
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, Throwable cause, BackupContext desc) {
+    super(msg, cause);
+    this.description = desc;
+  }
+
+  /**
+   * Exception when the description of the backup cannot be determined, due to some other root
+   * cause
+   * @param message description of what caused the failure
+   * @param e root cause
+   */
+  public BackupException(String message, Exception e) {
+    super(message, e);
+  }
+
+  public BackupContext getBackupContext() {
+    return this.description;
+  }
+
+}


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

Posted by en...@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-7912
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));


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

Posted by en...@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");


[07/50] [abbrv] hbase git commit: HBASE-15302 Reenable the other tests disabled by HBASE-14678

Posted by en...@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-7912
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


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

Posted by en...@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>

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

Posted by en...@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-7912
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()) {


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

Posted by en...@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>

[35/50] [abbrv] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/99955a32/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>

[29/50] [abbrv] hbase git commit: HBASE-15348 Disable metrics tests until fixed.

Posted by en...@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-7912
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;


[20/50] [abbrv] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by en...@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>

[33/50] [abbrv] hbase git commit: HBASE-15290 Hbase Rest CheckAndAPI should save other cells along with compared cell (Ajith)

Posted by en...@apache.org.
HBASE-15290 Hbase Rest CheckAndAPI should save other cells along with compared cell (Ajith)


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

Branch: refs/heads/HBASE-7912
Commit: 793babf4a4c0156f9e712a2bbf9578e2a1d6e1e4
Parents: 8f6e297
Author: Enis Soztutar <en...@apache.org>
Authored: Fri Feb 26 15:05:59 2016 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Fri Feb 26 15:05:59 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/rest/RowResource.java   | 32 +++++++--
 .../hadoop/hbase/rest/RowResourceBase.java      | 39 +++++++++--
 .../hbase/rest/TestGetAndPutResource.java       | 69 ++++++++++++++++++++
 3 files changed, 129 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/793babf4/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
index f922343..bac4edb 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
@@ -455,20 +455,40 @@ public class RowResource extends ResourceBase {
       byte[][] valueToPutParts = KeyValue.parseColumn(valueToCheckColumn);
       if (valueToPutParts.length == 2 && valueToPutParts[1].length > 0) {
         CellModel valueToPutCell = null;
+
+        // Copy all the cells to the Put request
+        // and track if the check cell's latest value is also sent
         for (int i = 0, n = cellModelCount - 1; i < n ; i++) {
-          if(Bytes.equals(cellModels.get(i).getColumn(),
-              valueToCheckCell.getColumn())) {
-            valueToPutCell = cellModels.get(i);
-            break;
+          CellModel cell = cellModels.get(i);
+          byte[] col = cell.getColumn();
+
+          if (col == null) {
+            servlet.getMetrics().incrementFailedPutRequests(1);
+            return Response.status(Response.Status.BAD_REQUEST)
+                    .type(MIMETYPE_TEXT).entity("Bad request: Column found to be null." + CRLF)
+                    .build();
+          }
+
+          byte [][] parts = KeyValue.parseColumn(col);
+
+          if (parts.length != 2) {
+            return Response.status(Response.Status.BAD_REQUEST)
+                    .type(MIMETYPE_TEXT).entity("Bad request" + CRLF)
+                    .build();
+          }
+          put.addImmutable(parts[0], parts[1], cell.getTimestamp(), cell.getValue());
+
+          if(Bytes.equals(col,
+                  valueToCheckCell.getColumn())) {
+            valueToPutCell = cell;
           }
         }
+
         if (valueToPutCell == null) {
           servlet.getMetrics().incrementFailedPutRequests(1);
           return Response.status(Response.Status.BAD_REQUEST).type(MIMETYPE_TEXT)
               .entity("Bad request: The column to put and check do not match." + CRLF).build();
         } else {
-          put.addImmutable(valueToPutParts[0], valueToPutParts[1], valueToPutCell.getTimestamp(),
-            valueToPutCell.getValue());
           retValue = table.checkAndPut(key, valueToPutParts[0], valueToPutParts[1],
             valueToCheckCell.getValue(), put);
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/793babf4/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
index 48cebb2..b2fc0a6 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.StringWriter;
+import java.util.*;
 
 import javax.ws.rs.core.MediaType;
 import javax.xml.bind.JAXBContext;
@@ -29,6 +30,7 @@ import javax.xml.bind.JAXBException;
 import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
 
+import org.apache.commons.collections.keyvalue.AbstractMapEntry;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -232,13 +234,22 @@ public class RowResourceBase {
   }
 
   protected static Response checkAndPutValuePB(String url, String table,
-      String row, String column, String valueToCheck, String valueToPut)
+      String row, String column, String valueToCheck, String valueToPut, HashMap<String,String> otherCells)
         throws IOException {
     RowModel rowModel = new RowModel(row);
     rowModel.addCell(new CellModel(Bytes.toBytes(column),
       Bytes.toBytes(valueToPut)));
+
+    if(otherCells != null) {
+      for (Map.Entry<String,String> entry :otherCells.entrySet()) {
+        rowModel.addCell(new CellModel(Bytes.toBytes(entry.getKey()), Bytes.toBytes(entry.getValue())));
+      }
+    }
+
+    // This Cell need to be added as last cell.
     rowModel.addCell(new CellModel(Bytes.toBytes(column),
       Bytes.toBytes(valueToCheck)));
+
     CellSetModel cellSetModel = new CellSetModel();
     cellSetModel.addRow(rowModel);
     Response response = client.put(url, Constants.MIMETYPE_PROTOBUF,
@@ -249,6 +260,10 @@ public class RowResourceBase {
 
   protected static Response checkAndPutValuePB(String table, String row,
       String column, String valueToCheck, String valueToPut) throws IOException {
+    return checkAndPutValuePB(table,row,column,valueToCheck,valueToPut,null);
+  }
+    protected static Response checkAndPutValuePB(String table, String row,
+      String column, String valueToCheck, String valueToPut, HashMap<String,String> otherCells) throws IOException {
     StringBuilder path = new StringBuilder();
     path.append('/');
     path.append(table);
@@ -256,15 +271,23 @@ public class RowResourceBase {
     path.append(row);
     path.append("?check=put");
     return checkAndPutValuePB(path.toString(), table, row, column,
-      valueToCheck, valueToPut);
+      valueToCheck, valueToPut, otherCells);
   }
 
   protected static Response checkAndPutValueXML(String url, String table,
-      String row, String column, String valueToCheck, String valueToPut)
+      String row, String column, String valueToCheck, String valueToPut, HashMap<String,String> otherCells)
         throws IOException, JAXBException {
     RowModel rowModel = new RowModel(row);
     rowModel.addCell(new CellModel(Bytes.toBytes(column),
       Bytes.toBytes(valueToPut)));
+
+    if(otherCells != null) {
+      for (Map.Entry<String,String> entry :otherCells.entrySet()) {
+        rowModel.addCell(new CellModel(Bytes.toBytes(entry.getKey()), Bytes.toBytes(entry.getValue())));
+      }
+    }
+
+    // This Cell need to be added as last cell.
     rowModel.addCell(new CellModel(Bytes.toBytes(column),
       Bytes.toBytes(valueToCheck)));
     CellSetModel cellSetModel = new CellSetModel();
@@ -278,7 +301,13 @@ public class RowResourceBase {
   }
 
   protected static Response checkAndPutValueXML(String table, String row,
-      String column, String valueToCheck, String valueToPut)
+                                                String column, String valueToCheck, String valueToPut)
+          throws IOException, JAXBException {
+    return checkAndPutValueXML(table,row,column,valueToCheck,valueToPut, null);
+  }
+
+  protected static Response checkAndPutValueXML(String table, String row,
+      String column, String valueToCheck, String valueToPut, HashMap<String,String> otherCells)
         throws IOException, JAXBException {
     StringBuilder path = new StringBuilder();
     path.append('/');
@@ -287,7 +316,7 @@ public class RowResourceBase {
     path.append(row);
     path.append("?check=put");
     return checkAndPutValueXML(path.toString(), table, row, column,
-      valueToCheck, valueToPut);
+      valueToCheck, valueToPut, otherCells);
   }
 
   protected static Response checkAndDeleteXML(String url, String table,

http://git-wip-us.apache.org/repos/asf/hbase/blob/793babf4/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
index a5326af..e778549 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
@@ -24,7 +24,10 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.net.URLEncoder;
+import java.util.Dictionary;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import javax.xml.bind.JAXBException;
 
@@ -129,6 +132,72 @@ public class TestGetAndPutResource extends RowResourceBase {
   }
 
   @Test
+  public void testMultipleCellCheckPutPB() throws IOException, JAXBException {
+    Response response = getValuePB(TABLE, ROW_1, COLUMN_1);
+    assertEquals(response.getCode(), 404);
+
+    // Add 2 Columns to setup the test
+    response = putValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+
+    response = putValuePB(TABLE, ROW_1, COLUMN_2, VALUE_2);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_2, VALUE_2);
+
+    HashMap<String,String> otherCells = new HashMap<String, String>();
+    otherCells.put(COLUMN_2,VALUE_3);
+
+    // On Success update both the cells
+    response = checkAndPutValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1, VALUE_3, otherCells);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_3);
+    checkValuePB(TABLE, ROW_1, COLUMN_2, VALUE_3);
+
+    // On Failure, we dont update any cells
+    response = checkAndPutValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1, VALUE_4, otherCells);
+    assertEquals(response.getCode(), 304);
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_3);
+    checkValuePB(TABLE, ROW_1, COLUMN_2, VALUE_3);
+
+    response = deleteRow(TABLE, ROW_1);
+    assertEquals(response.getCode(), 200);
+  }
+
+  @Test
+  public void testMultipleCellCheckPutXML() throws IOException, JAXBException {
+    Response response = getValuePB(TABLE, ROW_1, COLUMN_1);
+    assertEquals(response.getCode(), 404);
+
+    // Add 2 Columns to setup the test
+    response = putValueXML(TABLE, ROW_1, COLUMN_1, VALUE_1);
+    assertEquals(response.getCode(), 200);
+    checkValueXML(TABLE, ROW_1, COLUMN_1, VALUE_1);
+
+    response = putValueXML(TABLE, ROW_1, COLUMN_2, VALUE_2);
+    assertEquals(response.getCode(), 200);
+    checkValueXML(TABLE, ROW_1, COLUMN_2, VALUE_2);
+
+    HashMap<String,String> otherCells = new HashMap<String, String>();
+    otherCells.put(COLUMN_2,VALUE_3);
+
+    // On Success update both the cells
+    response = checkAndPutValueXML(TABLE, ROW_1, COLUMN_1, VALUE_1, VALUE_3, otherCells);
+    assertEquals(response.getCode(), 200);
+    checkValueXML(TABLE, ROW_1, COLUMN_1, VALUE_3);
+    checkValueXML(TABLE, ROW_1, COLUMN_2, VALUE_3);
+
+    // On Failure, we dont update any cells
+    response = checkAndPutValueXML(TABLE, ROW_1, COLUMN_1, VALUE_1, VALUE_4, otherCells);
+    assertEquals(response.getCode(), 304);
+    checkValueXML(TABLE, ROW_1, COLUMN_1, VALUE_3);
+    checkValueXML(TABLE, ROW_1, COLUMN_2, VALUE_3);
+
+    response = deleteRow(TABLE, ROW_1);
+    assertEquals(response.getCode(), 200);
+  }
+
+  @Test
   public void testSingleCellGetPutBinary() throws IOException {
     final String path = "/" + TABLE + "/" + ROW_3 + "/" + COLUMN_1;
     final byte[] body = Bytes.toBytes(VALUE_3);


[46/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
new file mode 100644
index 0000000..18a0f06
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -0,0 +1,571 @@
+/**
+ * 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.backup.impl;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+
+/**
+ * This class provides 'hbase:backup' table API
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupSystemTable implements Closeable {
+
+  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
+  private final static String TABLE_NAMESPACE = "hbase";
+  private final static String TABLE_NAME = "backup";
+  private final static TableName tableName = TableName.valueOf(TABLE_NAMESPACE, TABLE_NAME);
+  final static byte[] familyName = "f".getBytes();
+
+  // Connection to HBase cluster, shared
+  // among all instances
+  private final Connection connection;
+  // Cluster configuration
+  private final Configuration conf;
+
+  /**
+   * Create a BackupSystemTable object for the given Connection. Connection is NOT owned by this
+   * instance and has to be closed explicitly.
+   * @param connection
+   * @throws IOException
+   */
+  public BackupSystemTable(Connection connection) throws IOException {
+    this.connection = connection;
+    this.conf = connection.getConfiguration();
+
+    createSystemTableIfNotExists();
+  }
+
+  @Override
+  public void close() {
+  }
+
+  /**
+   * Gets table name
+   * @return table name
+   */
+  public static TableName getTableName() {
+    return tableName;
+  }
+
+  private void createSystemTableIfNotExists() throws IOException {
+    try(Admin admin = connection.getAdmin()) {
+      if (admin.tableExists(tableName) == false) {
+        HTableDescriptor tableDesc = new HTableDescriptor(tableName);
+        HColumnDescriptor colDesc = new HColumnDescriptor(familyName);
+        colDesc.setMaxVersions(1);
+        int ttl =
+            conf.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT);
+        colDesc.setTimeToLive(ttl);
+        tableDesc.addFamily(colDesc);
+        admin.createTable(tableDesc);
+      }
+    } catch (IOException e) {
+      LOG.error(e);
+      throw e;
+    }
+  }
+
+  /**
+   * Updates status (state) of a backup session in hbase:backup table
+   * @param context context
+   * @throws IOException exception
+   */
+  public void updateBackupStatus(BackupContext context) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("update backup status in hbase:backup for: " + context.getBackupId()
+        + " set status=" + context.getState());
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = BackupSystemTableHelper.createPutForBackupContext(context);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Deletes backup status from hbase:backup table
+   * @param backupId backup id
+   * @throws IOException exception
+   */
+
+  public void deleteBackupStatus(String backupId) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("delete backup status in hbase:backup for " + backupId);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Delete del = BackupSystemTableHelper.createDeletForBackupContext(backupId);
+      table.delete(del);
+    }
+  }
+
+  /**
+   * Reads backup status object (instance of BackupContext) from hbase:backup table
+   * @param backupId - backupId
+   * @return Current status of backup session or null
+   */
+
+  public BackupContext readBackupStatus(String backupId) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read backup status from hbase:backup for: " + backupId);
+    }
+
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForBackupContext(backupId);
+      Result res = table.get(get);
+      if(res.isEmpty()){
+        return null;
+      }
+      return BackupSystemTableHelper.resultToBackupContext(res);
+    }
+  }
+
+  /**
+   * Read the last backup start code (timestamp) of last successful backup. Will return null if
+   * there is no start code stored on hbase or the value is of length 0. These two cases indicate
+   * there is no successful backup completed so far.
+   * @return the timestamp of last successful backup
+   * @throws IOException exception
+   */
+  public String readBackupStartCode() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read backup start code from hbase:backup");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForStartCode();
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return null;
+      }
+      Cell cell = res.listCells().get(0);
+      byte[] val = CellUtil.cloneValue(cell);
+      if (val.length == 0){
+        return null;
+      }
+      return new String(val);
+    }
+  }
+
+  /**
+   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(Long startCode) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write backup start code to hbase:backup " + startCode);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = BackupSystemTableHelper.createPutForStartCode(startCode.toString());
+      table.put(put);
+    }
+  }
+
+  /**
+   * Get the Region Servers log information after the last log roll from hbase:backup.
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> readRegionServerLastLogRollResult()
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read region server last roll log result to hbase:backup");
+    }
+
+    Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult();
+    scan.setMaxVersions(1);
+
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      HashMap<String, Long> rsTimestampMap = new HashMap<String, Long>();
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        Cell cell = res.current();
+        byte[] row = CellUtil.cloneRow(cell);
+        String server =
+            BackupSystemTableHelper.getServerNameForReadRegionServerLastLogRollResult(row);
+
+        byte[] data = CellUtil.cloneValue(cell);
+        rsTimestampMap.put(server, Long.parseLong(new String(data)));
+      }
+      return rsTimestampMap;
+    }
+  }
+
+  /**
+   * Writes Region Server last roll log result (timestamp) to hbase:backup table
+   * @param server - Region Server name
+   * @param timestamp - last log timestamp
+   * @throws IOException exception
+   */
+  public void writeRegionServerLastLogRollResult(String server, Long timestamp)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write region server last roll log result to hbase:backup");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put =
+          BackupSystemTableHelper.createPutForRegionServerLastLogRollResult(server, timestamp);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public ArrayList<BackupCompleteData> getBackupHistory() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get backup history from hbase:backup");
+    }
+    Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+    scan.setMaxVersions(1);
+
+    ArrayList<BackupCompleteData> list = new ArrayList<BackupCompleteData>();
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current());
+        if (context.getState() != BackupState.COMPLETE) {
+          continue;
+        }
+
+        BackupCompleteData history = new BackupCompleteData();
+        history.setBackupToken(context.getBackupId());
+        history.setStartTime(Long.toString(context.getStartTs()));
+        history.setEndTime(Long.toString(context.getEndTs()));
+        history.setBackupRootPath(context.getTargetRootDir());
+        history.setTableList(context.getTableNames());
+        history.setType(context.getType().toString());
+        history.setBytesCopied(Long.toString(context.getTotalBytesCopied()));
+
+        list.add(history);
+      }
+      return BackupUtil.sortHistoryListDesc(list);
+    }
+  }
+
+  /**
+   * Get all backup session with a given status (in desc order by time)
+   * @param status status
+   * @return history info of backup contexts
+   * @throws IOException exception
+   */
+  public ArrayList<BackupContext> getBackupContexts(BackupState status) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get backup contexts from hbase:backup");
+    }
+
+    Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+    scan.setMaxVersions(1);
+    ArrayList<BackupContext> list = new ArrayList<BackupContext>();
+
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current());
+        if (context.getState() != status){
+          continue;
+        }
+        list.add(context);
+      }
+      return list;
+    }
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
+   * incremental backup. The saved timestamp is of the last log file that was backed up already.
+   * @param tables tables
+   * @param newTimestamps timestamps
+   * @throws IOException exception
+   */
+  public void writeRegionServerLogTimestamp(Set<TableName> tables,
+      HashMap<String, Long> newTimestamps) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write RS log ts to HBASE_BACKUP");
+    }
+    List<Put> puts = new ArrayList<Put>();
+    for (TableName table : tables) {
+      byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray();
+      Put put = BackupSystemTableHelper.createPutForWriteRegionServerLogTimestamp(table, smapData);
+      puts.add(put);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      table.put(puts);
+    }
+  }
+
+  /**
+   * Read the timestamp for each region server log after the last successful backup. Each table has
+   * its own set of the timestamps. The info is stored for each table as a concatenated string of
+   * rs->timestapmp
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<TableName, HashMap<String, Long>> readLogTimestampMap() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read RS log ts from HBASE_BACKUP");
+    }
+
+    HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
+        new HashMap<TableName, HashMap<String, Long>>();
+
+    Scan scan = BackupSystemTableHelper.createScanForReadLogTimestampMap();
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        Cell cell = res.current();
+        byte[] row = CellUtil.cloneRow(cell);
+        String tabName = BackupSystemTableHelper.getTableNameForReadLogTimestampMap(row);
+        TableName tn = TableName.valueOf(tabName);
+        HashMap<String, Long> lastBackup = new HashMap<String, Long>();
+        byte[] data = CellUtil.cloneValue(cell);
+
+        if (data == null) {
+          throw new IOException("Data of last backup data from HBASE_BACKUP "
+              + "is empty. Create a backup first.");
+        }
+        if (data != null && data.length > 0) {
+          lastBackup =
+              fromTableServerTimestampProto(BackupProtos.TableServerTimestamp.parseFrom(data));
+          tableTimestampMap.put(tn, lastBackup);
+        }
+      }
+      return tableTimestampMap;
+    }
+  }
+
+  private BackupProtos.TableServerTimestamp toTableServerTimestampProto(TableName table,
+      Map<String, Long> map) {
+    BackupProtos.TableServerTimestamp.Builder tstBuilder =
+        BackupProtos.TableServerTimestamp.newBuilder();
+    tstBuilder.setTable(ProtobufUtil.toProtoTableName(table));
+
+    for(Entry<String, Long> entry: map.entrySet()) {
+      BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder();
+      builder.setServer(entry.getKey());
+      builder.setTimestamp(entry.getValue());
+      tstBuilder.addServerTimestamp(builder.build());
+    }
+
+    return tstBuilder.build();
+  }
+
+  private HashMap<String, Long> fromTableServerTimestampProto(
+      BackupProtos.TableServerTimestamp proto) {
+    HashMap<String, Long> map = new HashMap<String, Long> ();
+    List<BackupProtos.ServerTimestamp> list = proto.getServerTimestampList();
+    for(BackupProtos.ServerTimestamp st: list) {
+      map.put(st.getServer(), st.getTimestamp());
+    }
+    return map;
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<TableName> getIncrementalBackupTableSet() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get incr backup table set from hbase:backup");
+    }
+    TreeSet<TableName> set = new TreeSet<>();
+
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForIncrBackupTableSet();
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return set;
+      }
+      List<Cell> cells = res.listCells();
+      for (Cell cell : cells) {
+        // qualifier = table name - we use table names as qualifiers
+        set.add(TableName.valueOf(CellUtil.cloneQualifier(cell)));
+      }
+      return set;
+    }
+  }
+
+  /**
+   * Add tables to global incremental backup set
+   * @param tables - set of tables
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<TableName> tables) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("add incr backup table set to hbase:backup");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = BackupSystemTableHelper.createPutForIncrBackupTableSet(tables);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Register WAL files as eligible for deletion
+   * @param files files
+   * @throws IOException exception
+   */
+  public void addWALFiles(List<String> files, String backupId) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("add WAL files to hbase:backup");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      List<Put> puts = BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId);
+      table.put(puts);
+    }
+  }
+
+  /**
+   * Register WAL files as eligible for deletion
+   * @param files files
+   * @throws IOException exception
+   */
+  public Iterator<String> getWALFilesIterator() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get WAL files from hbase:backup");
+    }
+    final Table table = connection.getTable(tableName);
+    Scan scan = BackupSystemTableHelper.createScanForGetWALs();
+    final ResultScanner scanner = table.getScanner(scan);
+    final Iterator<Result> it = scanner.iterator();
+    return new Iterator<String>() {
+
+      @Override
+      public boolean hasNext() {
+        boolean next = it.hasNext();
+        if (!next) {
+          // close all
+          try {
+            scanner.close();
+            table.close();
+          } catch (Exception e) {
+            LOG.error(e);
+          }
+        }
+        return next;
+      }
+
+      @Override
+      public String next() {
+        Result next = it.next();
+        List<Cell> cells = next.listCells();
+        byte[] buf = cells.get(0).getValueArray();
+        int len = cells.get(0).getValueLength();
+        int offset = cells.get(0).getValueOffset();
+        return new String(buf, offset, len);
+      }
+
+      @Override
+      public void remove() {
+        // not implemented
+        throw new RuntimeException("remove is not supported");
+      }
+    };
+
+  }
+
+  /**
+   * Check if WAL file is eligible for deletion
+   * @param file file
+   * @return true, if - yes.
+   * @throws IOException exception
+   */
+  public boolean checkWALFile(String file) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Check if WAL file has been already backuped in hbase:backup");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForCheckWALFile(file);
+      Result res = table.get(get);
+      if (res.isEmpty()){
+        return false;
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Checks if we have at least one backup session in hbase:backup This API is used by
+   * BackupLogCleaner
+   * @return true, if - at least one session exists in hbase:backup table
+   * @throws IOException exception
+   */
+  public boolean hasBackupSessions() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("has backup sessions from hbase:backup");
+    }
+    boolean result = false;
+    Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+    scan.setMaxVersions(1);
+    scan.setCaching(1);
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      if (scanner.next() != null) {
+        result = true;
+      }
+      return result;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
new file mode 100644
index 0000000..ac096b7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
@@ -0,0 +1,325 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * A collection for methods used by BackupSystemTable.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupSystemTableHelper {
+
+  /**
+   * hbase:backup schema:
+   * 1. Backup sessions rowkey= "session." + backupId; value = serialized
+   * BackupContext
+   * 2. Backup start code rowkey = "startcode"; value = startcode
+   * 3. Incremental backup set rowkey="incrbackupset"; value=[list of tables]
+   * 4. Table-RS-timestamp map rowkey="trslm"+ table_name; value = map[RS-> last WAL timestamp]
+   * 5. RS - WAL ts map rowkey="rslogts."+server; value = last WAL timestamp
+   * 6. WALs recorded rowkey="wals."+WAL unique file name; value = backuppId and full WAL file name
+   */
+
+  private final static String BACKUP_CONTEXT_PREFIX = "session.";
+  private final static String START_CODE_ROW = "startcode";
+  private final static String INCR_BACKUP_SET = "incrbackupset";
+  private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm.";
+  private final static String RS_LOG_TS_PREFIX = "rslogts.";
+  private final static String WALS_PREFIX = "wals.";
+
+  private final static byte[] col1 = "col1".getBytes();
+  private final static byte[] col2 = "col2".getBytes();
+
+  private final static byte[] EMPTY_VALUE = new byte[] {};
+
+  private BackupSystemTableHelper() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Creates Put operation for a given backup context object
+   * @param context backup context
+   * @return put operation
+   * @throws IOException exception
+   */
+  static Put createPutForBackupContext(BackupContext context) throws IOException {
+
+    Put put = new Put((BACKUP_CONTEXT_PREFIX + context.getBackupId()).getBytes());
+    put.addColumn(BackupSystemTable.familyName, col1, context.toByteArray());
+    return put;
+  }
+
+  /**
+   * Creates Get operation for a given backup id
+   * @param backupId - backup's ID
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForBackupContext(String backupId) throws IOException {
+    Get get = new Get((BACKUP_CONTEXT_PREFIX + backupId).getBytes());
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Delete operation for a given backup id
+   * @param backupId - backup's ID
+   * @return delete operation
+   * @throws IOException exception
+   */
+  public static Delete createDeletForBackupContext(String backupId) {
+    Delete del = new Delete((BACKUP_CONTEXT_PREFIX + backupId).getBytes());
+    del.addFamily(BackupSystemTable.familyName);
+    return del;
+  }
+
+  /**
+   * Converts Result to BackupContext
+   * @param res - HBase result
+   * @return backup context instance
+   * @throws IOException exception
+   */
+  static BackupContext resultToBackupContext(Result res) throws IOException {
+    res.advance();
+    Cell cell = res.current();
+    return cellToBackupContext(cell);
+  }
+
+  /**
+   * Creates Get operation to retrieve start code from hbase:backup
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForStartCode() throws IOException {
+    Get get = new Get(START_CODE_ROW.getBytes());
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put operation to store start code to hbase:backup
+   * @return put operation
+   * @throws IOException exception
+   */
+  static Put createPutForStartCode(String startCode) {
+    Put put = new Put(START_CODE_ROW.getBytes());
+    put.addColumn(BackupSystemTable.familyName, col1, startCode.getBytes());
+    return put;
+  }
+
+  /**
+   * Creates Get to retrieve incremental backup table set from hbase:backup
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForIncrBackupTableSet() throws IOException {
+    Get get = new Get(INCR_BACKUP_SET.getBytes());
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put to store incremental backup table set
+   * @param tables tables
+   * @return put operation
+   */
+  static Put createPutForIncrBackupTableSet(Set<TableName> tables) {
+    Put put = new Put(INCR_BACKUP_SET.getBytes());
+    for (TableName table : tables) {
+      put.addColumn(BackupSystemTable.familyName, Bytes.toBytes(table.getNameAsString()),
+        EMPTY_VALUE);
+    }
+    return put;
+  }
+
+  /**
+   * Creates Scan operation to load backup history
+   * @return scan operation
+   */
+  static Scan createScanForBackupHistory() {
+    Scan scan = new Scan();
+    byte[] startRow = BACKUP_CONTEXT_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.familyName);
+
+    return scan;
+  }
+
+  /**
+   * Converts cell to backup context instance.
+   * @param current - cell
+   * @return backup context instance
+   * @throws IOException exception
+   */
+  static BackupContext cellToBackupContext(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneValue(current);
+    return BackupContext.fromByteArray(data);
+  }
+
+  /**
+   * Creates Put to write RS last roll log timestamp map
+   * @param table - table
+   * @param smap - map, containing RS:ts
+   * @return put operation
+   */
+  static Put createPutForWriteRegionServerLogTimestamp(TableName table, byte[] smap) {
+    Put put = new Put((TABLE_RS_LOG_MAP_PREFIX + table).getBytes());
+    put.addColumn(BackupSystemTable.familyName, col1, smap);
+    return put;
+  }
+
+  /**
+   * Creates Scan to load table-> { RS -> ts} map of maps
+   * @return scan operation
+   */
+  static Scan createScanForReadLogTimestampMap() {
+    Scan scan = new Scan();
+    byte[] startRow = TABLE_RS_LOG_MAP_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.familyName);
+
+    return scan;
+  }
+
+  /**
+   * Get table name from rowkey
+   * @param cloneRow rowkey
+   * @return table name
+   */
+  static String getTableNameForReadLogTimestampMap(byte[] cloneRow) {
+    int prefixSize = TABLE_RS_LOG_MAP_PREFIX.length();
+    return new String(cloneRow, prefixSize, cloneRow.length - prefixSize);
+  }
+
+  /**
+   * Creates Put to store RS last log result
+   * @param server - server name
+   * @param timestamp - log roll result (timestamp)
+   * @return put operation
+   */
+  static Put createPutForRegionServerLastLogRollResult(String server, Long timestamp) {
+    Put put = new Put((RS_LOG_TS_PREFIX + server).getBytes());
+    put.addColumn(BackupSystemTable.familyName, col1, timestamp.toString().getBytes());
+    return put;
+  }
+
+  /**
+   * Creates Scan operation to load last RS log roll results
+   * @return scan operation
+   */
+  static Scan createScanForReadRegionServerLastLogRollResult() {
+    Scan scan = new Scan();
+    byte[] startRow = RS_LOG_TS_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.familyName);
+
+    return scan;
+  }
+
+  /**
+   * Get server's name from rowkey
+   * @param row - rowkey
+   * @return server's name
+   */
+  static String getServerNameForReadRegionServerLastLogRollResult(byte[] row) {
+    int prefixSize = RS_LOG_TS_PREFIX.length();
+    return new String(row, prefixSize, row.length - prefixSize);
+  }
+
+  /**
+   * Creates put list for list of WAL files
+   * @param files list of WAL file paths
+   * @param backupId backup id
+   * @return put list
+   * @throws IOException exception
+   */
+  public static List<Put> createPutsForAddWALFiles(List<String> files, String backupId)
+      throws IOException {
+
+    List<Put> puts = new ArrayList<Put>();
+    for (String file : files) {
+      byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes();
+      Put put = new Put(row);
+      put.addColumn(BackupSystemTable.familyName, col1, backupId.getBytes());
+      put.addColumn(BackupSystemTable.familyName, col2, file.getBytes());
+      puts.add(put);
+    }
+    return puts;
+  }
+
+  /**
+   * Creates Scan operation to load WALs
+   * @return scan operation
+   */
+  public static Scan createScanForGetWALs() {
+    Scan scan = new Scan();
+    byte[] startRow = WALS_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addColumn(BackupSystemTable.familyName, col2);
+    return scan;
+  }
+  /**
+   * Creates Get operation for a given wal file name
+   * @param file file
+   * @return get operation
+   * @throws IOException exception
+   */
+  public static Get createGetForCheckWALFile(String file) throws IOException {
+    byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes();
+    Get get = new Get(row);
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java
new file mode 100644
index 0000000..9981f78
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java
@@ -0,0 +1,453 @@
+/**
+ * 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.backup.impl;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+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.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+
+/**
+ * A collection for methods used by multiple classes to backup HBase tables.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupUtil {
+  protected static final Log LOG = LogFactory.getLog(BackupUtil.class);
+  public static final String LOGNAME_SEPARATOR = ".";
+
+  private BackupUtil(){
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Loop through the RS log timestamp map for the tables, for each RS, find the min timestamp
+   * value for the RS among the tables.
+   * @param rsLogTimestampMap timestamp map
+   * @return the min timestamp of each RS
+   */
+  protected static HashMap<String, Long> getRSLogTimestampMins(
+    HashMap<TableName, HashMap<String, Long>> rsLogTimestampMap) {
+
+    if (rsLogTimestampMap == null || rsLogTimestampMap.isEmpty()) {
+      return null;
+    }
+
+    HashMap<String, Long> rsLogTimestamptMins = new HashMap<String, Long>();
+    HashMap<String, HashMap<TableName, Long>> rsLogTimestampMapByRS =
+        new HashMap<String, HashMap<TableName, Long>>();
+
+    for (Entry<TableName, HashMap<String, Long>> tableEntry : rsLogTimestampMap.entrySet()) {
+      TableName table = tableEntry.getKey();
+      HashMap<String, Long> rsLogTimestamp = tableEntry.getValue();
+      for (Entry<String, Long> rsEntry : rsLogTimestamp.entrySet()) {
+        String rs = rsEntry.getKey();
+        Long ts = rsEntry.getValue();
+        if (!rsLogTimestampMapByRS.containsKey(rs)) {
+          rsLogTimestampMapByRS.put(rs, new HashMap<TableName, Long>());
+          rsLogTimestampMapByRS.get(rs).put(table, ts);
+        } else {
+          rsLogTimestampMapByRS.get(rs).put(table, ts);
+        }
+      }
+    }
+
+    for (String rs : rsLogTimestampMapByRS.keySet()) {
+      rsLogTimestamptMins.put(rs, BackupUtility.getMinValue(rsLogTimestampMapByRS.get(rs)));
+    }
+
+    return rsLogTimestamptMins;
+  }
+
+  /**
+   * copy out Table RegionInfo into incremental backup image need to consider move this logic into
+   * HBackupFileSystem
+   * @param backupContext backup context
+   * @param conf configuration
+   * @throws IOException exception
+   * @throws InterruptedException exception
+   */
+  protected static void copyTableRegionInfo(BackupContext backupContext, Configuration conf)
+      throws IOException, InterruptedException {
+
+    Path rootDir = FSUtils.getRootDir(conf);
+    FileSystem fs = rootDir.getFileSystem(conf);
+
+    // for each table in the table set, copy out the table info and region info files in the correct
+    // directory structure
+    for (TableName table : backupContext.getTables()) {
+
+      LOG.debug("Attempting to copy table info for:" + table);
+      TableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
+
+      // write a copy of descriptor to the target directory
+      Path target = new Path(backupContext.getBackupStatus(table).getTargetDir());
+      FileSystem targetFs = target.getFileSystem(conf);
+      FSTableDescriptors descriptors =
+          new FSTableDescriptors(conf, targetFs, FSUtils.getRootDir(conf));
+      descriptors.createTableDescriptorForTableDirectory(target, orig, false);
+      LOG.debug("Finished copying tableinfo.");
+
+      HBaseAdmin hbadmin = null;
+      // TODO: optimize
+      List<HRegionInfo> regions = null;
+      try(Connection conn = ConnectionFactory.createConnection(conf);
+          Admin admin = conn.getAdmin()) {
+        regions = admin.getTableRegions(table);
+      } catch (Exception e) {
+        throw new BackupException(e);
+      }
+
+      // For each region, write the region info to disk
+      LOG.debug("Starting to write region info for table " + table);
+      for (HRegionInfo regionInfo : regions) {
+        Path regionDir =
+            HRegion.getRegionDir(new Path(backupContext.getBackupStatus(table).getTargetDir()),
+              regionInfo);
+        regionDir =
+            new Path(backupContext.getBackupStatus(table).getTargetDir(), regionDir.getName());
+        writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo);
+      }
+      LOG.debug("Finished writing region info for table " + table);
+    }
+  }
+
+  /**
+   * Write the .regioninfo file on-disk.
+   */
+  public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs,
+      final Path regionInfoDir, HRegionInfo regionInfo) throws IOException {
+    final byte[] content = regionInfo.toDelimitedByteArray();
+    Path regionInfoFile = new Path(regionInfoDir, ".regioninfo");
+    // First check to get the permissions
+    FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
+    // Write the RegionInfo file content
+    FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null);
+    try {
+      out.write(content);
+    } finally {
+      out.close();
+    }
+  }
+
+  /**
+   * TODO: return hostname:port
+   * @param p
+   * @return host name: port
+   * @throws IOException
+   */
+  public static String parseHostNameFromLogFile(Path p) throws IOException {
+    if (isArchivedLogFile(p)) {
+      return BackupUtility.parseHostFromOldLog(p);
+    } else {
+      ServerName sname = DefaultWALProvider.getServerNameFromWALDirectoryName(p);
+      return sname.getHostname() + ":" + sname.getPort();
+    }
+  }
+
+  private static boolean isArchivedLogFile(Path p) {
+    String oldLog = Path.SEPARATOR + HConstants.HREGION_OLDLOGDIR_NAME + Path.SEPARATOR;
+    return p.toString().contains(oldLog);
+  }
+
+  /**
+   * Returns WAL file name
+   * @param walFileName WAL file name
+   * @return WAL file name
+   * @throws IOException exception
+   * @throws IllegalArgumentException exception
+   */
+  public static String getUniqueWALFileNamePart(String walFileName) throws IOException {
+    return getUniqueWALFileNamePart(new Path(walFileName));
+  }
+
+  /**
+   * Returns WAL file name
+   * @param p - WAL file path
+   * @return WAL file name
+   * @throws IOException exception
+   */
+  public static String getUniqueWALFileNamePart(Path p) throws IOException {
+    return p.getName();
+  }
+
+  /**
+   * Get the total length of files under the given directory recursively.
+   * @param fs The hadoop file system
+   * @param dir The target directory
+   * @return the total length of files
+   * @throws IOException exception
+   */
+  public static long getFilesLength(FileSystem fs, Path dir) throws IOException {
+    long totalLength = 0;
+    FileStatus[] files = FSUtils.listStatus(fs, dir);
+    if (files != null) {
+      for (FileStatus fileStatus : files) {
+        if (fileStatus.isDirectory()) {
+          totalLength += getFilesLength(fs, fileStatus.getPath());
+        } else {
+          totalLength += fileStatus.getLen();
+        }
+      }
+    }
+    return totalLength;
+  }
+
+  /**
+   * Keep the record for dependency for incremental backup and history info p.s, we may be able to
+   * merge this class into backupImage class later
+   */
+  public static class BackupCompleteData implements Comparable<BackupCompleteData> {
+    private String startTime;
+    private String endTime;
+    private String type;
+    private String backupRootPath;
+    private List<TableName> tableList;
+    private String backupToken;
+    private String bytesCopied;
+    private List<String> ancestors;
+
+    public List<String> getAncestors() {
+      if (this.ancestors == null) {
+        this.ancestors = new ArrayList<String>();
+      }
+      return this.ancestors;
+    }
+
+    public void addAncestor(String backupToken) {
+      this.getAncestors().add(backupToken);
+    }
+
+    public String getBytesCopied() {
+      return bytesCopied;
+    }
+
+    public void setBytesCopied(String bytesCopied) {
+      this.bytesCopied = bytesCopied;
+    }
+
+    public String getBackupToken() {
+      return backupToken;
+    }
+
+    public void setBackupToken(String backupToken) {
+      this.backupToken = backupToken;
+    }
+
+    public String getStartTime() {
+      return startTime;
+    }
+
+    public void setStartTime(String startTime) {
+      this.startTime = startTime;
+    }
+
+    public String getEndTime() {
+      return endTime;
+    }
+
+    public void setEndTime(String endTime) {
+      this.endTime = endTime;
+    }
+
+    public String getType() {
+      return type;
+    }
+
+    public void setType(String type) {
+      this.type = type;
+    }
+
+    public String getBackupRootPath() {
+      return backupRootPath;
+    }
+
+    public void setBackupRootPath(String backupRootPath) {
+      this.backupRootPath = backupRootPath;
+    }
+
+    public List<TableName> getTableList() {
+      return tableList;
+    }
+
+    public void setTableList(List<TableName> tableList) {
+      this.tableList = tableList;
+    }
+
+    @Override
+    public int compareTo(BackupCompleteData o) {
+      Long thisTS =
+          new Long(this.getBackupToken().substring(this.getBackupToken().lastIndexOf("_") + 1));
+      Long otherTS =
+          new Long(o.getBackupToken().substring(o.getBackupToken().lastIndexOf("_") + 1));
+      return thisTS.compareTo(otherTS);
+    }
+
+  }
+
+  /**
+   * Sort history list by start time in descending order.
+   * @param historyList history list
+   * @return sorted list of BackupCompleteData
+   */
+  public static ArrayList<BackupCompleteData> sortHistoryListDesc(
+    ArrayList<BackupCompleteData> historyList) {
+    ArrayList<BackupCompleteData> list = new ArrayList<BackupCompleteData>();
+    TreeMap<String, BackupCompleteData> map = new TreeMap<String, BackupCompleteData>();
+    for (BackupCompleteData h : historyList) {
+      map.put(h.getStartTime(), h);
+    }
+    Iterator<String> i = map.descendingKeySet().iterator();
+    while (i.hasNext()) {
+      list.add(map.get(i.next()));
+    }
+    return list;
+  }
+
+  /**
+   * Get list of all WAL files (WALs and archive)
+   * @param c - configuration
+   * @return list of WAL files
+   * @throws IOException exception
+   */
+  public static List<String> getListOfWALFiles(Configuration c) throws IOException {
+    Path rootDir = FSUtils.getRootDir(c);
+    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    List<String> logFiles = new ArrayList<String>();
+
+    FileSystem fs = FileSystem.get(c);
+    logFiles = BackupUtility.getFiles(fs, logDir, logFiles, null);
+    logFiles = BackupUtility.getFiles(fs, oldLogDir, logFiles, null);
+    return logFiles;
+  }
+
+  /**
+   * Get list of all WAL files (WALs and archive)
+   * @param c - configuration
+   * @return list of WAL files
+   * @throws IOException exception
+   */
+  public static List<String> getListOfWALFiles(Configuration c, PathFilter filter)
+      throws IOException {
+    Path rootDir = FSUtils.getRootDir(c);
+    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    List<String> logFiles = new ArrayList<String>();
+
+    FileSystem fs = FileSystem.get(c);
+    logFiles = BackupUtility.getFiles(fs, logDir, logFiles, filter);
+    logFiles = BackupUtility.getFiles(fs, oldLogDir, logFiles, filter);
+    return logFiles;
+  }
+
+  /**
+   * Get list of all old WAL files (WALs and archive)
+   * @param c - configuration
+   * @return list of WAL files
+   * @throws IOException exception
+   */
+  public static List<String> getWALFilesOlderThan(final Configuration c,
+    final HashMap<String, Long> hostTimestampMap) throws IOException {
+    Path rootDir = FSUtils.getRootDir(c);
+    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    List<String> logFiles = new ArrayList<String>();
+
+    PathFilter filter = new PathFilter() {
+
+      @Override
+      public boolean accept(Path p) {
+        try {
+          if (DefaultWALProvider.isMetaFile(p)) {
+            return false;
+          }
+          String host = parseHostNameFromLogFile(p);
+          Long oldTimestamp = hostTimestampMap.get(host);
+          Long currentLogTS = BackupUtility.getCreationTime(p);
+          return currentLogTS <= oldTimestamp;
+        } catch (IOException e) {
+          LOG.error(e);
+          return false;
+        }
+      }
+    };
+    FileSystem fs = FileSystem.get(c);
+    logFiles = BackupUtility.getFiles(fs, logDir, logFiles, filter);
+    logFiles = BackupUtility.getFiles(fs, oldLogDir, logFiles, filter);
+    return logFiles;
+  }
+
+  public static String join(TableName[] names) {
+    StringBuilder sb = new StringBuilder();
+    String sep = BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND;
+    for (TableName s : names) {
+      sb.append(sep).append(s.getNameAsString());
+    }
+    return sb.toString();
+  }
+
+  public static TableName[] parseTableNames(String tables) {
+    if (tables == null) {
+      return null;
+    }
+    String[] tableArray = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+
+    TableName[] ret = new TableName[tableArray.length];
+    for (int i = 0; i < tableArray.length; i++) {
+      ret[i] = TableName.valueOf(tableArray[i]);
+    }
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
new file mode 100644
index 0000000..a3d124f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
@@ -0,0 +1,304 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+
+/**
+ * After a full backup was created, the incremental backup will only store the changes made
+ * after the last full or incremental backup.
+ *
+ * Creating the backup copies the logfiles in .logs and .oldlogs since the last backup timestamp.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class IncrementalBackupManager {
+  public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class);
+
+  // parent manager
+  private final BackupManager backupManager;
+  private final Configuration conf;
+  private final Connection conn;
+
+  public IncrementalBackupManager(BackupManager bm) {
+    this.backupManager = bm;
+    this.conf = bm.getConf();
+    this.conn = bm.getConnection();
+  }
+
+  /**
+   * Obtain the list of logs that need to be copied out for this incremental backup. The list is set
+   * in BackupContext.
+   * @param backupContext backup context
+   * @return The new HashMap of RS log timestamps after the log roll for this incremental backup.
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> getIncrBackupLogFileList(BackupContext backupContext)
+      throws IOException {
+    List<String> logList;
+    HashMap<String, Long> newTimestamps;
+    HashMap<String, Long> previousTimestampMins;
+
+    String savedStartCode = backupManager.readBackupStartCode();
+
+    // key: tableName
+    // value: <RegionServer,PreviousTimeStamp>
+    HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
+        backupManager.readLogTimestampMap();
+
+    previousTimestampMins = BackupUtil.getRSLogTimestampMins(previousTimestampMap);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("StartCode " + savedStartCode + "for backupID " + backupContext.getBackupId());
+    }
+    // get all new log files from .logs and .oldlogs after last TS and before new timestamp
+    if (savedStartCode == null ||
+        previousTimestampMins == null ||
+          previousTimestampMins.isEmpty()) {
+      throw new IOException("Cannot read any previous back up timestamps from hbase:backup. "
+          + "In order to create an incremental backup, at least one full backup is needed.");
+    }
+
+    try (Admin admin = conn.getAdmin()) {
+      LOG.info("Execute roll log procedure for incremental backup ...");
+      admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap<String, String>());
+    }
+
+    newTimestamps = backupManager.readRegionServerLastLogRollResult();
+
+    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
+    logList.addAll(getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps));
+    backupContext.setIncrBackupFileList(logList);
+
+    return newTimestamps;
+  }
+
+  /**
+   * For each region server: get all log files newer than the last timestamps but not newer than the
+   * newest timestamps. FROM hbase:backup table
+   * @param olderTimestamps - the timestamp for each region server of the last backup.
+   * @param newestTimestamps - the timestamp for each region server that the backup should lead to.
+   * @return list of log files which needs to be added to this backup
+   * @throws IOException
+   */
+  private List<String> getLogFilesFromBackupSystem(HashMap<String, Long> olderTimestamps,
+    HashMap<String, Long> newestTimestamps) throws IOException {
+    List<String> logFiles = new ArrayList<String>();
+    Iterator<String> it = backupManager.getWALFilesFromBackupSystem();
+
+    while (it.hasNext()) {
+      String walFileName = it.next();
+      String server = BackupUtil.parseHostNameFromLogFile(new Path(walFileName));
+      //String server = getServer(walFileName);
+      Long tss = getTimestamp(walFileName);
+      Long oldTss = olderTimestamps.get(server);
+      if (oldTss == null){
+        logFiles.add(walFileName);
+        continue;
+      }
+      Long newTss = newestTimestamps.get(server);
+      if (newTss == null) {
+        newTss = Long.MAX_VALUE;
+      }
+
+      if (tss > oldTss && tss < newTss) {
+        logFiles.add(walFileName);
+      }
+    }
+    return logFiles;
+  }
+
+  private Long getTimestamp(String walFileName) {
+    int index = walFileName.lastIndexOf(BackupUtil.LOGNAME_SEPARATOR);
+    return Long.parseLong(walFileName.substring(index+1));
+  }
+
+  /**
+   * For each region server: get all log files newer than the last timestamps but not newer than the
+   * newest timestamps.
+   * @param olderTimestamps the timestamp for each region server of the last backup.
+   * @param newestTimestamps the timestamp for each region server that the backup should lead to.
+   * @param conf the Hadoop and Hbase configuration
+   * @param savedStartCode the startcode (timestamp) of last successful backup.
+   * @return a list of log files to be backed up
+   * @throws IOException exception
+   */
+  private List<String> getLogFilesForNewBackup(HashMap<String, Long> olderTimestamps,
+    HashMap<String, Long> newestTimestamps, Configuration conf, String savedStartCode)
+        throws IOException {
+    LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps
+      + "\n newestTimestamps: " + newestTimestamps);
+    Path rootdir = FSUtils.getRootDir(conf);
+    Path logDir = new Path(rootdir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
+    FileSystem fs = rootdir.getFileSystem(conf);
+    NewestLogFilter pathFilter = new NewestLogFilter();
+
+    List<String> resultLogFiles = new ArrayList<String>();
+    List<String> newestLogs = new ArrayList<String>();
+
+    /*
+     * The old region servers and timestamps info we kept in hbase:backup may be out of sync if new
+     * region server is added or existing one lost. We'll deal with it here when processing the
+     * logs. If data in hbase:backup has more hosts, just ignore it. If the .logs directory includes
+     * more hosts, the additional hosts will not have old timestamps to compare with. We'll just use
+     * all the logs in that directory. We always write up-to-date region server and timestamp info
+     * to hbase:backup at the end of successful backup.
+     */
+
+    FileStatus[] rss;
+    Path p;
+    String host;
+    Long oldTimeStamp;
+    String currentLogFile;
+    Long currentLogTS;
+
+    // Get the files in .logs.
+    rss = fs.listStatus(logDir);
+    for (FileStatus rs : rss) {
+      p = rs.getPath();
+      host = BackupUtil.parseHostNameFromLogFile(p);
+      FileStatus[] logs;
+      oldTimeStamp = olderTimestamps.get(host);
+      // It is possible that there is no old timestamp in hbase:backup for this host if
+      // this region server is newly added after our last backup.
+      if (oldTimeStamp == null) {
+        logs = fs.listStatus(p);
+      } else {
+        pathFilter.setLastBackupTS(oldTimeStamp);
+        logs = fs.listStatus(p, pathFilter);
+      }
+      for (FileStatus log : logs) {
+        LOG.debug("currentLogFile: " + log.getPath().toString());
+        if (DefaultWALProvider.isMetaFile(log.getPath())) {
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Skip hbase:meta log file: " + log.getPath().getName());
+          }
+          continue;
+        }
+        currentLogFile = log.getPath().toString();
+        resultLogFiles.add(currentLogFile);
+        currentLogTS = BackupUtility.getCreationTime(log.getPath());
+        // newestTimestamps is up-to-date with the current list of hosts
+        // so newestTimestamps.get(host) will not be null.
+        if (Long.valueOf(currentLogTS) > Long.valueOf(newestTimestamps.get(host))) {
+          newestLogs.add(currentLogFile);
+        }
+      }
+    }
+
+    // Include the .oldlogs files too.
+    FileStatus[] oldlogs = fs.listStatus(oldLogDir);
+    for (FileStatus oldlog : oldlogs) {
+      p = oldlog.getPath();
+      currentLogFile = p.toString();
+      if (DefaultWALProvider.isMetaFile(p)) {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Skip .meta log file: " + currentLogFile);
+        }
+        continue;
+      }
+      host = BackupUtility.parseHostFromOldLog(p);
+      currentLogTS = BackupUtility.getCreationTime(p);
+      oldTimeStamp = olderTimestamps.get(host);
+      /*
+       * It is possible that there is no old timestamp in hbase:backup for this host. At the time of
+       * our last backup operation, this rs did not exist. The reason can be one of the two: 1. The
+       * rs already left/crashed. Its logs were moved to .oldlogs. 2. The rs was added after our
+       * last backup.
+       */
+      if (oldTimeStamp == null) {
+        if (Long.valueOf(currentLogTS) < Long.valueOf(savedStartCode)) {
+          // This log file is really old, its region server was before our last backup.
+          continue;
+        } else {
+          resultLogFiles.add(currentLogFile);
+        }
+      } else if (Long.valueOf(currentLogTS) > Long.valueOf(oldTimeStamp)) {
+        resultLogFiles.add(currentLogFile);
+      }
+
+      // It is possible that a host in .oldlogs is an obsolete region server
+      // so newestTimestamps.get(host) here can be null.
+      // Even if these logs belong to a obsolete region server, we still need
+      // to include they to avoid loss of edits for backup.
+      Long newTimestamp = newestTimestamps.get(host);
+      if (newTimestamp != null && Long.valueOf(currentLogTS) > Long.valueOf(newTimestamp)) {
+        newestLogs.add(currentLogFile);
+      }
+    }
+    // remove newest log per host because they are still in use
+    resultLogFiles.removeAll(newestLogs);
+    return resultLogFiles;
+  }
+
+  class NewestLogFilter implements PathFilter {
+    private Long lastBackupTS = 0L;
+
+    public NewestLogFilter() {
+    }
+
+    protected void setLastBackupTS(Long ts) {
+      this.lastBackupTS = ts;
+    }
+
+    @Override
+    public boolean accept(Path path) {
+      // skip meta table log -- ts.meta file
+      if (DefaultWALProvider.isMetaFile(path)) {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Skip .meta log file: " + path.getName());
+        }
+        return false;
+      }
+      Long timestamp = null;
+      try {
+        timestamp = BackupUtility.getCreationTime(path);
+        return timestamp > Long.valueOf(lastBackupTS);
+      } catch (IOException e) {
+        LOG.warn("Cannot read timestamp of log file " + path);
+        return false;
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java
new file mode 100644
index 0000000..12ecbe9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalRestoreService.java
@@ -0,0 +1,34 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface IncrementalRestoreService extends Configurable{
+
+  public void run(String logDirectory, TableName[] fromTables, TableName[] toTables)
+    throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
new file mode 100644
index 0000000..2f8ab1b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
@@ -0,0 +1,313 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.RestoreClient;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+
+/**
+ * The main class which interprets the given arguments and trigger restore operation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class RestoreClientImpl implements RestoreClient {
+
+  private static final Log LOG = LogFactory.getLog(RestoreClientImpl.class);
+  private Configuration conf;
+  private Set<BackupImage> lastRestoreImagesSet;
+
+  public RestoreClientImpl() {
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Restore operation. Stage 1: validate backupManifest, and check target tables
+   * @param backupRootDir The root dir for backup image
+   * @param backupId The backup id for image to be restored
+   * @param check True if only do dependency check
+   * @param autoRestore True if automatically restore following the dependency
+   * @param sTableArray The array of tables to be restored
+   * @param tTableArray The array of mapping tables to restore to
+   * @param isOverwrite True then do restore overwrite if target table exists, otherwise fail the
+   *          request if target table exists
+   * @return True if only do dependency check
+   * @throws IOException if any failure during restore
+   */
+  @Override
+  public boolean restore(String backupRootDir,
+      String backupId, boolean check, boolean autoRestore, TableName[] sTableArray,
+      TableName[] tTableArray, boolean isOverwrite) throws IOException {
+
+    HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
+    // check and load backup image manifest for the tables
+    Path rootPath = new Path(backupRootDir);
+    HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath,
+      backupId);
+
+    try {
+      // Check and validate the backup image and its dependencies
+      if (check || autoRestore) {
+        if (validate(backupManifestMap)) {
+          LOG.info("Checking backup images: ok");
+        } else {
+          String errMsg = "Some dependencies are missing for restore";
+          LOG.error(errMsg);
+          throw new IOException(errMsg);
+        }
+      }
+
+      // return true if only for check
+      if (check) {
+        return true;
+      }
+
+      if (tTableArray == null) {
+        tTableArray = sTableArray;
+      }
+
+      // check the target tables
+      checkTargetTables(tTableArray, isOverwrite);
+
+      // start restore process
+      Set<BackupImage> restoreImageSet =
+          restoreStage(backupManifestMap, sTableArray, tTableArray, autoRestore);
+
+      LOG.info("Restore for " + Arrays.asList(sTableArray) + " are successful!");
+      lastRestoreImagesSet = restoreImageSet;
+
+    } catch (IOException e) {
+      LOG.error("ERROR: restore failed with error: " + e.getMessage());
+      throw e;
+    }
+
+    // not only for check, return false
+    return false;
+  }
+
+  /**
+   * Get last restore image set. The value is globally set for the latest finished restore.
+   * @return the last restore image set
+   */
+  public Set<BackupImage> getLastRestoreImagesSet() {
+    return lastRestoreImagesSet;
+  }
+
+  private  boolean validate(HashMap<TableName, BackupManifest> backupManifestMap)
+      throws IOException {
+    boolean isValid = true;
+
+    for (Entry<TableName, BackupManifest> manifestEntry : backupManifestMap.entrySet()) {
+      TableName table = manifestEntry.getKey();
+      TreeSet<BackupImage> imageSet = new TreeSet<BackupImage>();
+
+      ArrayList<BackupImage> depList = manifestEntry.getValue().getDependentListByTable(table);
+      if (depList != null && !depList.isEmpty()) {
+        imageSet.addAll(depList);
+      }
+
+      // todo merge
+      LOG.debug("merge will be implemented in future jira");
+      // BackupUtil.clearMergedImages(table, imageSet, conf);
+
+      LOG.info("Dependent image(s) from old to new:");
+      for (BackupImage image : imageSet) {
+        String imageDir =
+            HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table);
+        if (!BackupUtility.checkPathExist(imageDir, conf)) {
+          LOG.error("ERROR: backup image does not exist: " + imageDir);
+          isValid = false;
+          break;
+        }
+        // TODO More validation?
+        LOG.info("Backup image: " + image.getBackupId() + " for '" + table + "' is available");
+      }
+    }
+    return isValid;
+  }
+
+  /**
+   * Validate target Tables
+   * @param tTableArray: target tables
+   * @param isOverwrite overwrite existing table
+   * @throws IOException exception
+   */
+  private  void checkTargetTables(TableName[] tTableArray, boolean isOverwrite)
+      throws IOException {
+    ArrayList<TableName> existTableList = new ArrayList<>();
+    ArrayList<TableName> disabledTableList = new ArrayList<>();
+
+    // check if the tables already exist
+    try(Connection conn = ConnectionFactory.createConnection(conf);
+        Admin admin = conn.getAdmin()) {
+      for (TableName tableName : tTableArray) {
+        if (admin.tableExists(tableName)) {
+          existTableList.add(tableName);
+          if (admin.isTableDisabled(tableName)) {
+            disabledTableList.add(tableName);
+          }
+        } else {
+          LOG.info("HBase table " + tableName
+              + " does not exist. It will be create during backup process");
+        }
+      }
+    }
+
+    if (existTableList.size() > 0) {
+      if (!isOverwrite) {
+        LOG.error("Existing table found in the restore target, please add \"-overwrite\" "
+            + "option in the command if you mean to restore to these existing tables");
+        LOG.info("Existing table list in restore target: " + existTableList);
+        throw new IOException("Existing table found in target while no \"-overwrite\" "
+            + "option found");
+      } else {
+        if (disabledTableList.size() > 0) {
+          LOG.error("Found offline table in the restore target, "
+              + "please enable them before restore with \"-overwrite\" option");
+          LOG.info("Offline table list in restore target: " + disabledTableList);
+          throw new IOException(
+              "Found offline table in the target when restore with \"-overwrite\" option");
+        }
+      }
+    }
+
+  }
+
+  /**
+   * Restore operation. Stage 2: resolved Backup Image dependency
+   * @param backupManifestMap : tableName,  Manifest
+   * @param sTableArray The array of tables to be restored
+   * @param tTableArray The array of mapping tables to restore to
+   * @param autoRestore : yes, restore all the backup images on the dependency list
+   * @return set of BackupImages restored
+   * @throws IOException exception
+   */
+  private Set<BackupImage> restoreStage(
+    HashMap<TableName, BackupManifest> backupManifestMap, TableName[] sTableArray,
+    TableName[] tTableArray, boolean autoRestore) throws IOException {
+    TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
+
+    for (int i = 0; i < sTableArray.length; i++) {
+      restoreImageSet.clear();
+      TableName table = sTableArray[i];
+      BackupManifest manifest = backupManifestMap.get(table);
+      if (autoRestore) {
+        // Get the image list of this backup for restore in time order from old
+        // to new.
+        TreeSet<BackupImage> restoreList =
+            new TreeSet<BackupImage>(manifest.getDependentListByTable(table));
+        LOG.debug("need to clear merged Image. to be implemented in future jira");
+
+        for (BackupImage image : restoreList) {
+          restoreImage(image, table, tTableArray[i]);
+        }
+        restoreImageSet.addAll(restoreList);
+      } else {
+        BackupImage image = manifest.getBackupImage();
+        List<BackupImage> depList = manifest.getDependentListByTable(table);
+        // The dependency list always contains self.
+        if (depList != null && depList.size() > 1) {
+          LOG.warn("Backup image " + image.getBackupId() + " depends on other images.\n"
+              + "this operation will only restore the delta contained within backupImage "
+              + image.getBackupId());
+        }
+        restoreImage(image, table, tTableArray[i]);
+        restoreImageSet.add(image);
+      }
+
+      if (autoRestore) {
+        if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
+          LOG.info("Restore includes the following image(s):");
+          for (BackupImage image : restoreImageSet) {
+            LOG.info("  Backup: "
+                + image.getBackupId()
+                + " "
+                + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(),
+                  table));
+          }
+        }
+      }
+
+    }
+    return restoreImageSet;
+  }
+
+  /**
+   * Restore operation handle each backupImage
+   * @param image: backupImage
+   * @param sTable: table to be restored
+   * @param tTable: table to be restored to
+   * @throws IOException exception
+   */
+  private  void restoreImage(BackupImage image, TableName sTable, TableName tTable)
+      throws IOException {
+    String rootDir = image.getRootDir();
+    String backupId = image.getBackupId();
+
+    Path rootPath = new Path(rootDir);
+    HBackupFileSystem hFS = new HBackupFileSystem(conf, rootPath, backupId);
+    RestoreUtil restoreTool = new RestoreUtil(conf, hFS);
+    BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, rootPath, backupId);
+
+    Path tableBackupPath = HBackupFileSystem.getTableBackupPath(rootPath, sTable, backupId);
+
+    // todo: convert feature will be provided in a future jira
+    boolean converted = false;
+
+    if (manifest.getType() == BackupType.FULL || converted) {
+      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from "
+          + (converted ? "converted" : "full") + " backup image " + tableBackupPath.toString());
+      restoreTool.fullRestoreTable(tableBackupPath, sTable, tTable, converted);
+    } else { // incremental Backup
+      String logBackupDir =
+          HBackupFileSystem.getLogBackupDir(image.getRootDir(), image.getBackupId());
+      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from incremental backup image "
+          + logBackupDir);
+      restoreTool.incrementalRestoreTable(logBackupDir, new TableName[] { sTable },
+        new TableName[] { tTable });
+    }
+
+    LOG.info(sTable + " has been successfully restored to " + tTable);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java
new file mode 100644
index 0000000..3882e95
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreUtil.java
@@ -0,0 +1,298 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+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.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+
+/**
+ * A collection for methods used by multiple classes to restore HBase tables.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RestoreUtil {
+
+  public static final Log LOG = LogFactory.getLog(RestoreUtil.class);
+
+  protected Configuration conf = null;
+
+  protected HBackupFileSystem hBackupFS = null;
+
+  protected Path backupRootPath;
+
+  protected String backupId;
+
+  // store table name and snapshot dir mapping
+  private final HashMap<TableName, Path> snapshotMap = new HashMap<>();
+
+  public RestoreUtil(Configuration conf, HBackupFileSystem hBackupFS) throws IOException {
+    this.conf = conf;
+    this.hBackupFS = hBackupFS;
+    this.backupRootPath = hBackupFS.getBackupRootPath();
+    this.backupId = hBackupFS.getBackupId();
+  }
+
+  /**
+   * During incremental backup operation. Call WalPlayer to replay WAL in backup image Currently
+   * tableNames and newTablesNames only contain single table, will be expanded to multiple tables in
+   * the future
+   * @param logDir : incremental backup folders, which contains WAL
+   * @param tableNames : source tableNames(table names were backuped)
+   * @param newTableNames : target tableNames(table names to be restored to)
+   * @throws IOException exception
+   */
+  public void incrementalRestoreTable(String logDir,
+      TableName[] tableNames, TableName[] newTableNames) throws IOException {
+
+    if (tableNames.length != newTableNames.length) {
+      throw new IOException("Number of source tables adn taget Tables does not match!");
+    }
+
+    // for incremental backup image, expect the table already created either by user or previous
+    // full backup. Here, check that all new tables exists
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+        Admin admin = conn.getAdmin()) {
+      for (TableName tableName : newTableNames) {
+        if (!admin.tableExists(tableName)) {
+          admin.close();
+          throw new IOException("HBase table " + tableName
+            + " does not exist. Create the table first, e.g. by restoring a full backup.");
+        }
+      }
+      IncrementalRestoreService restoreService =
+          BackupRestoreFactory.getIncrementalRestoreService(conf);
+
+      restoreService.run(logDir, tableNames, newTableNames);
+    }
+  }
+
+  public void fullRestoreTable(Path tableBackupPath, TableName tableName, TableName newTableName,
+      boolean converted) throws IOException {
+    restoreTableAndCreate(tableName, newTableName, tableBackupPath, converted);
+  }
+
+  private void restoreTableAndCreate(TableName tableName, TableName newTableName,
+      Path tableBackupPath, boolean converted) throws IOException {
+    if (newTableName == null || newTableName.equals("")) {
+      newTableName = tableName;
+    }
+
+    FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
+
+    // get table descriptor first
+    HTableDescriptor tableDescriptor = null;
+
+    Path tableSnapshotPath = HBackupFileSystem.getTableSnapshotPath(backupRootPath, tableName,
+      backupId);
+
+    if (fileSys.exists(tableSnapshotPath)) {
+      // snapshot path exist means the backup path is in HDFS
+      // check whether snapshot dir already recorded for target table
+      if (snapshotMap.get(tableName) != null) {
+        SnapshotDescription desc =
+            SnapshotDescriptionUtils.readSnapshotInfo(fileSys, tableSnapshotPath);
+        SnapshotManifest manifest = SnapshotManifest.open(conf, fileSys, tableSnapshotPath, desc);
+        tableDescriptor = manifest.getTableDescriptor();
+
+
+      } else {
+        tableDescriptor = hBackupFS.getTableDesc(tableName);
+        snapshotMap.put(tableName, hBackupFS.getTableInfoPath(tableName));
+      }
+      if (tableDescriptor == null) {
+        LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost");
+      }
+    } else if (converted) {
+      // first check if this is a converted backup image
+      LOG.error("convert will be supported in a future jira");
+    }
+
+    Path tableArchivePath = hBackupFS.getTableArchivePath(tableName);
+    if (tableArchivePath == null) {
+      if (tableDescriptor != null) {
+        // find table descriptor but no archive dir means the table is empty, create table and exit
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("find table descriptor but no archive dir for table " + tableName
+            + ", will only create table");
+        }
+        tableDescriptor.setName(newTableName);
+        checkAndCreateTable(tableBackupPath, tableName, newTableName, null, tableDescriptor);
+        return;
+      } else {
+        throw new IllegalStateException("Cannot restore hbase table because directory '"
+            + " tableArchivePath is null.");
+      }
+    }
+
+    if (tableDescriptor == null) {
+      tableDescriptor = new HTableDescriptor(newTableName);
+    } else {
+      tableDescriptor.setName(newTableName);
+    }
+
+    if (!converted) {
+      // record all region dirs:
+      // load all files in dir
+      try {
+        ArrayList<Path> regionPathList = hBackupFS.getRegionList(tableName);
+
+        // should only try to create the table with all region informations, so we could pre-split
+        // the regions in fine grain
+        checkAndCreateTable(tableBackupPath, tableName, newTableName, regionPathList,
+          tableDescriptor);
+        if (tableArchivePath != null) {
+          // start real restore through bulkload
+          // if the backup target is on local cluster, special action needed
+          Path tempTableArchivePath = hBackupFS.checkLocalAndBackup(tableArchivePath);
+          if (tempTableArchivePath.equals(tableArchivePath)) {
+            if(LOG.isDebugEnabled()) {
+              LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath);
+            }
+          } else {
+            regionPathList = hBackupFS.getRegionList(tempTableArchivePath); // point to the tempDir
+            if(LOG.isDebugEnabled()) {
+              LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath);
+            }
+          }
+
+          LoadIncrementalHFiles loader = createLoader(tempTableArchivePath, false);
+          for (Path regionPath : regionPathList) {
+            String regionName = regionPath.toString();
+            if(LOG.isDebugEnabled()) {
+              LOG.debug("Restoring HFiles from directory " + regionName);
+            }
+            String[] args = { regionName, newTableName.getNameAsString()};
+            loader.run(args);
+          }
+        }
+        // we do not recovered edits
+      } catch (Exception e) {
+        throw new IllegalStateException("Cannot restore hbase table", e);
+      }
+    } else {
+      LOG.debug("convert will be supported in a future jira");
+    }
+  }
+
+
+
+  /**
+   * Create a {@link LoadIncrementalHFiles} instance to be used to restore the HFiles of a full
+   * backup.
+   * @return the {@link LoadIncrementalHFiles} instance
+   * @throws IOException exception
+   */
+  private LoadIncrementalHFiles createLoader(Path tableArchivePath, boolean multipleTables)
+      throws IOException {
+    // set configuration for restore:
+    // LoadIncrementalHFile needs more time
+    // <name>hbase.rpc.timeout</name> <value>600000</value>
+    // calculates
+    Integer milliSecInMin = 60000;
+    Integer previousMillis = this.conf.getInt("hbase.rpc.timeout", 0);
+    Integer numberOfFilesInDir =
+        multipleTables ? hBackupFS.getMaxNumberOfFilesInSubDir(tableArchivePath) : hBackupFS
+            .getNumberOfFilesInDir(tableArchivePath);
+    Integer calculatedMillis = numberOfFilesInDir * milliSecInMin; // 1 minute per file
+    Integer resultMillis = Math.max(calculatedMillis, previousMillis);
+    if (resultMillis > previousMillis) {
+      LOG.info("Setting configuration for restore with LoadIncrementalHFile: "
+          + "hbase.rpc.timeout to " + calculatedMillis / milliSecInMin
+          + " minutes, to handle the number of files in backup " + tableArchivePath);
+      this.conf.setInt("hbase.rpc.timeout", resultMillis);
+    }
+
+    LoadIncrementalHFiles loader = null;
+    try {
+      loader = new LoadIncrementalHFiles(this.conf);
+    } catch (Exception e1) {
+      throw new IOException(e1);
+    }
+    return loader;
+  }
+
+  /**
+   * Prepare the table for bulkload, most codes copied from
+   * {@link LoadIncrementalHFiles#createTable(String, String)}
+   * @param tableBackupPath path
+   * @param tableName table name
+   * @param targetTableName target table name
+   * @param regionDirList region directory list
+   * @param htd table descriptor
+   * @throws IOException exception
+   */
+  private void checkAndCreateTable(Path tableBackupPath, TableName tableName,
+      TableName targetTableName, ArrayList<Path> regionDirList, HTableDescriptor htd)
+          throws IOException {
+    HBaseAdmin hbadmin = null;
+    Connection conn = null;
+    try {
+      conn = ConnectionFactory.createConnection(conf);
+      hbadmin = (HBaseAdmin) conn.getAdmin();
+      if (hbadmin.tableExists(targetTableName)) {
+        LOG.info("Using exising target table '" + targetTableName + "'");
+      } else {
+        LOG.info("Creating target table '" + targetTableName + "'");
+
+        // if no region dir given, create the table and return
+        if (regionDirList == null || regionDirList.size() == 0) {
+
+          hbadmin.createTable(htd);
+          return;
+        }
+
+        byte[][] keys = hBackupFS.generateBoundaryKeys(regionDirList);
+
+        // create table using table decriptor and region boundaries
+        hbadmin.createTable(htd, keys);
+      }
+    } catch (Exception e) {
+      throw new IOException(e);
+    } finally {
+      if (hbadmin != null) {
+        hbadmin.close();
+      }
+      if(conn != null){
+        conn.close();
+      }
+    }
+  }
+
+}


[42/50] [abbrv] hbase git commit: HBASE-15346 add 1.3 RM to docs

Posted by en...@apache.org.
HBASE-15346 add 1.3 RM to docs


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

Branch: refs/heads/HBASE-7912
Commit: bc112888ef557e7d6bf0cc9e91c0d6d63d4f89f9
Parents: 83297f6
Author: Mikhail Antonov <an...@apache.org>
Authored: Sun Feb 28 17:12:32 2016 -0800
Committer: Mikhail Antonov <an...@apache.org>
Committed: Mon Feb 29 11:06:10 2016 -0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/developer.adoc | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bc112888/src/main/asciidoc/_chapters/developer.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
index 4238bc2..f8473cd 100644
--- a/src/main/asciidoc/_chapters/developer.adoc
+++ b/src/main/asciidoc/_chapters/developer.adoc
@@ -1362,6 +1362,9 @@ NOTE: End-of-life releases are not included in this list.
 | 1.2
 | Sean Busbey
 
+| 1.3
+| Mikhail Antonov
+
 |===
 
 [[code.standards]]


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

Posted by en...@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-7912
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");


[40/50] [abbrv] hbase git commit: HBASE-15358 canEnforceTimeLimitFromScope should use timeScope instead of sizeScope

Posted by en...@apache.org.
HBASE-15358 canEnforceTimeLimitFromScope should use timeScope instead of sizeScope

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-7912
Commit: 88f775996b3b52d784ad13ab07515134619316ba
Parents: 03ffb30
Author: Phil Yang <ud...@gmail.com>
Authored: Mon Feb 29 16:26:51 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 29 17:21:42 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/ScannerContext.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/88f77599/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
index 2062230..6674443 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
@@ -626,7 +626,7 @@ public class ScannerContext {
      * @return true when the limit can be enforced from the scope of the checker
      */
     boolean canEnforceTimeLimitFromScope(LimitScope checkerScope) {
-      return this.sizeScope.canEnforceLimitFromScope(checkerScope);
+      return this.timeScope.canEnforceLimitFromScope(checkerScope);
     }
 
     @Override


[43/50] [abbrv] hbase git commit: HBASE-15323 Hbase Rest CheckAndDeleteAPi should be able to delete more cells (Ajith)

Posted by en...@apache.org.
HBASE-15323 Hbase Rest CheckAndDeleteAPi should be able to delete more cells (Ajith)


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

Branch: refs/heads/HBASE-7912
Commit: 7c54525c89bbbe0c66401813433bfb957e461eac
Parents: bc11288
Author: Enis Soztutar <en...@apache.org>
Authored: Mon Feb 29 12:24:18 2016 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Feb 29 12:24:18 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/rest/RowResource.java   | 52 ++++++++++++++++---
 .../hadoop/hbase/rest/client/RemoteHTable.java  |  1 +
 .../hadoop/hbase/rest/RowResourceBase.java      | 49 +++++++++++++++---
 .../hbase/rest/TestGetAndPutResource.java       | 53 ++++++++++++++++++++
 4 files changed, 142 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7c54525c/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
index bac4edb..15828ce 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
@@ -553,9 +553,12 @@ public class RowResource extends ResourceBase {
           .build();
       }
 
+      List<CellModel> cellModels = rowModel.getCells();
+      int cellModelCount = cellModels.size();
+
       delete = new Delete(key);
       boolean retValue;
-      CellModel valueToDeleteCell = rowModel.getCells().get(0);
+      CellModel valueToDeleteCell = rowModel.getCells().get(cellModelCount -1);
       byte[] valueToDeleteColumn = valueToDeleteCell.getColumn();
       if (valueToDeleteColumn == null) {
         try {
@@ -567,25 +570,62 @@ public class RowResource extends ResourceBase {
             .build();
         }
       }
-      byte[][] parts = KeyValue.parseColumn(valueToDeleteColumn);
+
+      byte[][] parts ;
+      // Copy all the cells to the Delete request if extra cells are sent
+      if(cellModelCount > 1) {
+        for (int i = 0, n = cellModelCount - 1; i < n; i++) {
+          CellModel cell = cellModels.get(i);
+          byte[] col = cell.getColumn();
+
+          if (col == null) {
+            servlet.getMetrics().incrementFailedPutRequests(1);
+            return Response.status(Response.Status.BAD_REQUEST)
+                    .type(MIMETYPE_TEXT).entity("Bad request: Column found to be null." + CRLF)
+                    .build();
+          }
+
+          parts = KeyValue.parseColumn(col);
+
+          if (parts.length == 1) {
+            // Only Column Family is specified
+            delete.addFamily(parts[0], cell.getTimestamp());
+          } else if (parts.length == 2) {
+            delete.addColumn(parts[0], parts[1], cell.getTimestamp());
+          } else {
+            servlet.getMetrics().incrementFailedDeleteRequests(1);
+            return Response.status(Response.Status.BAD_REQUEST)
+                    .type(MIMETYPE_TEXT)
+                    .entity("Bad request: Column to delete incorrectly specified." + CRLF)
+                    .build();
+          }
+        }
+      }
+
+      parts = KeyValue.parseColumn(valueToDeleteColumn);
       if (parts.length == 2) {
         if (parts[1].length != 0) {
-          delete.addColumns(parts[0], parts[1]);
+          // To support backcompat of deleting a cell
+          // if that is the only cell passed to the rest api
+          if(cellModelCount == 1) {
+            delete.addColumns(parts[0], parts[1]);
+          }
           retValue = table.checkAndDelete(key, parts[0], parts[1],
             valueToDeleteCell.getValue(), delete);
         } else {
           // The case of empty qualifier.
-          delete.addColumns(parts[0], Bytes.toBytes(StringUtils.EMPTY));
+          if(cellModelCount == 1) {
+            delete.addColumns(parts[0], Bytes.toBytes(StringUtils.EMPTY));
+          }
           retValue = table.checkAndDelete(key, parts[0], Bytes.toBytes(StringUtils.EMPTY),
             valueToDeleteCell.getValue(), delete);
         }
       } else {
         servlet.getMetrics().incrementFailedDeleteRequests(1);
         return Response.status(Response.Status.BAD_REQUEST)
-          .type(MIMETYPE_TEXT).entity("Bad request: Column incorrectly specified." + CRLF)
+          .type(MIMETYPE_TEXT).entity("Bad request: Column to check incorrectly specified." + CRLF)
           .build();
       }
-      delete.addColumns(parts[0], parts[1]);
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("CHECK-AND-DELETE " + delete.toString() + ", returns "

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c54525c/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
index 172b763..2a30e99 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
@@ -722,6 +722,7 @@ public class RemoteHTable implements Table {
   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Delete delete) throws IOException {
     Put put = new Put(row);
+    put.setFamilyCellMap(delete.getFamilyCellMap());
     // column to check-the-value
     put.add(new KeyValue(row, family, qualifier, value));
     CellSetModel model = buildModelFromPut(put);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c54525c/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
index b2fc0a6..61e650b 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
@@ -320,9 +320,16 @@ public class RowResourceBase {
   }
 
   protected static Response checkAndDeleteXML(String url, String table,
-      String row, String column, String valueToCheck)
+      String row, String column, String valueToCheck, HashMap<String,String> cellsToDelete)
         throws IOException, JAXBException {
     RowModel rowModel = new RowModel(row);
+
+    if(cellsToDelete != null) {
+      for (Map.Entry<String,String> entry :cellsToDelete.entrySet()) {
+        rowModel.addCell(new CellModel(Bytes.toBytes(entry.getKey()), Bytes.toBytes(entry.getValue())));
+      }
+    }
+    // Add this at the end
     rowModel.addCell(new CellModel(Bytes.toBytes(column),
       Bytes.toBytes(valueToCheck)));
     CellSetModel cellSetModel = new CellSetModel();
@@ -337,30 +344,46 @@ public class RowResourceBase {
 
   protected static Response checkAndDeleteXML(String table, String row,
       String column, String valueToCheck) throws IOException, JAXBException {
+    return checkAndDeleteXML(table, row, column, valueToCheck, null);
+  }
+  protected static Response checkAndDeleteXML(String table, String row,
+      String column, String valueToCheck, HashMap<String,String> cellsToDelete) throws IOException, JAXBException {
     StringBuilder path = new StringBuilder();
     path.append('/');
     path.append(table);
     path.append('/');
     path.append(row);
     path.append("?check=delete");
-    return checkAndDeleteXML(path.toString(), table, row, column, valueToCheck);
+    return checkAndDeleteXML(path.toString(), table, row, column, valueToCheck, cellsToDelete);
   }
 
   protected static Response checkAndDeleteJson(String table, String row,
       String column, String valueToCheck) throws IOException, JAXBException {
+    return checkAndDeleteJson(table, row, column, valueToCheck, null);
+  }
+
+  protected static Response checkAndDeleteJson(String table, String row,
+      String column, String valueToCheck, HashMap<String,String> cellsToDelete) throws IOException, JAXBException {
     StringBuilder path = new StringBuilder();
     path.append('/');
     path.append(table);
     path.append('/');
     path.append(row);
     path.append("?check=delete");
-    return checkAndDeleteJson(path.toString(), table, row, column, valueToCheck);
+    return checkAndDeleteJson(path.toString(), table, row, column, valueToCheck, cellsToDelete);
   }
 
   protected static Response checkAndDeleteJson(String url, String table,
-      String row, String column, String valueToCheck)
+      String row, String column, String valueToCheck, HashMap<String,String> cellsToDelete)
         throws IOException, JAXBException {
     RowModel rowModel = new RowModel(row);
+
+    if(cellsToDelete != null) {
+      for (Map.Entry<String,String> entry :cellsToDelete.entrySet()) {
+        rowModel.addCell(new CellModel(Bytes.toBytes(entry.getKey()), Bytes.toBytes(entry.getValue())));
+      }
+    }
+    // Add this at the end
     rowModel.addCell(new CellModel(Bytes.toBytes(column),
       Bytes.toBytes(valueToCheck)));
     CellSetModel cellSetModel = new CellSetModel();
@@ -374,19 +397,31 @@ public class RowResourceBase {
 
   protected static Response checkAndDeletePB(String table, String row,
       String column, String value) throws IOException {
+
+    return checkAndDeletePB(table, row, column, value, null);
+  }
+
+  protected static Response checkAndDeletePB(String table, String row,
+      String column, String value, HashMap<String,String> cellsToDelete) throws IOException {
     StringBuilder path = new StringBuilder();
     path.append('/');
     path.append(table);
     path.append('/');
     path.append(row);
     path.append("?check=delete");
-    return checkAndDeleteValuePB(path.toString(), table, row, column, value);
+    return checkAndDeleteValuePB(path.toString(), table, row, column, value, cellsToDelete);
   }
-
   protected static Response checkAndDeleteValuePB(String url, String table,
-      String row, String column, String valueToCheck)
+      String row, String column, String valueToCheck, HashMap<String,String> cellsToDelete)
       throws IOException {
     RowModel rowModel = new RowModel(row);
+
+    if(cellsToDelete != null) {
+      for (Map.Entry<String,String> entry :cellsToDelete.entrySet()) {
+        rowModel.addCell(new CellModel(Bytes.toBytes(entry.getKey()), Bytes.toBytes(entry.getValue())));
+      }
+    }
+    // Add this at the end
     rowModel.addCell(new CellModel(Bytes.toBytes(column), Bytes
         .toBytes(valueToCheck)));
     CellSetModel cellSetModel = new CellSetModel();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c54525c/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
index e778549..c6fb2ff 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
@@ -198,6 +198,59 @@ public class TestGetAndPutResource extends RowResourceBase {
   }
 
   @Test
+  public void testMultipleCellCheckDeletePB() throws IOException, JAXBException {
+    Response response = getValuePB(TABLE, ROW_1, COLUMN_1);
+    assertEquals(response.getCode(), 404);
+
+    // Add 3 Columns to setup the test
+    response = putValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+
+    response = putValuePB(TABLE, ROW_1, COLUMN_2, VALUE_2);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_2, VALUE_2);
+
+    response = putValuePB(TABLE, ROW_1, COLUMN_3, VALUE_3);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_3, VALUE_3);
+
+    // Deletes the following columns based on Column1 check
+    HashMap<String,String> cellsToDelete = new HashMap<String, String>();
+    cellsToDelete.put(COLUMN_2,VALUE_2); // Value does not matter
+    cellsToDelete.put(COLUMN_3,VALUE_3); // Value does not matter
+
+    // On Success update both the cells
+    response = checkAndDeletePB(TABLE, ROW_1, COLUMN_1, VALUE_1, cellsToDelete);
+    assertEquals(response.getCode(), 200);
+
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+
+    response = getValuePB(TABLE, ROW_1, COLUMN_2);
+    assertEquals(response.getCode(), 404);
+
+    response = getValuePB(TABLE, ROW_1, COLUMN_3);
+    assertEquals(response.getCode(), 404);
+
+    response = putValuePB(TABLE, ROW_1, COLUMN_2, VALUE_2);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_2, VALUE_2);
+
+    response = putValuePB(TABLE, ROW_1, COLUMN_3, VALUE_3);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_3, VALUE_3);
+
+    // On Failure, we dont update any cells
+    response = checkAndDeletePB(TABLE, ROW_1, COLUMN_1, VALUE_3, cellsToDelete);
+    assertEquals(response.getCode(), 304);
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+    checkValuePB(TABLE, ROW_1, COLUMN_2, VALUE_2);
+    checkValuePB(TABLE, ROW_1, COLUMN_3, VALUE_3);
+
+    response = deleteRow(TABLE, ROW_1);
+    assertEquals(response.getCode(), 200);
+  }
+  @Test
   public void testSingleCellGetPutBinary() throws IOException {
     final String path = "/" + TABLE + "/" + ROW_3 + "/" + COLUMN_1;
     final byte[] body = Bytes.toBytes(VALUE_3);


[38/50] [abbrv] hbase git commit: HBASE-15181 A simple implementation of date based tiered compaction (Clara Xiong)

Posted by en...@apache.org.
HBASE-15181 A simple implementation of date based tiered compaction (Clara Xiong)


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

Branch: refs/heads/HBASE-7912
Commit: f7f96b9fb70f5b2243558cf531ab7fa51162e656
Parents: 99955a3
Author: tedyu <yu...@gmail.com>
Authored: Fri Feb 26 17:36:23 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Feb 26 17:36:23 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/StoreFile.java    |   7 +
 .../compactions/CompactionConfiguration.java    |  82 +++++-
 .../compactions/DateTieredCompactionPolicy.java | 294 +++++++++++++++++++
 .../compactions/RatioBasedCompactionPolicy.java |  18 +-
 .../hbase/regionserver/MockStoreFile.java       |  12 +
 .../regionserver/TestDateTieredCompaction.java  | 211 +++++++++++++
 .../TestDefaultCompactSelection.java            | 187 +-----------
 7 files changed, 622 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f7f96b9f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 4ced556..61eb9b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -768,6 +768,13 @@ public class StoreFile {
         getReader().timeRangeTracker.getMinimumTimestamp();
   }
 
+  public Long getMaximumTimestamp() {
+    return (getReader().timeRangeTracker == null) ?
+        null :
+        getReader().timeRangeTracker.getMaximumTimestamp();
+  }
+
+
   /**
    * Gets the approximate mid-point of this file that is optimal for use in splitting it.
    * @param comparator Comparator used to compare KVs.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f7f96b9f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
index 633477e..9bb4c77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
@@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
 
 /**
@@ -67,6 +67,23 @@ public class CompactionConfiguration {
   public static final String HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT =
       "hbase.hfile.compaction.discharger.thread.count";
 
+  /*
+   * The epoch time length for the windows we no longer compact
+   */
+  public static final String MAX_AGE_MILLIS_KEY =
+    "hbase.hstore.compaction.date.tiered.max.storefile.age.millis";
+  public static final String BASE_WINDOW_MILLIS_KEY =
+    "hbase.hstore.compaction.date.tiered.base.window.millis";
+  public static final String WINDOWS_PER_TIER_KEY =
+    "hbase.hstore.compaction.date.tiered.windows.per.tier";
+  public static final String INCOMING_WINDOW_MIN_KEY =
+    "hbase.hstore.compaction.date.tiered.incoming.window.min";
+  public static final String COMPACTION_POLICY_CLASS_FOR_TIERED_WINDOWS_KEY =
+    "hbase.hstore.compaction.date.tiered.window.policy.class";
+
+  private static final Class<? extends RatioBasedCompactionPolicy>
+    DEFAULT_TIER_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class;
+
   Configuration conf;
   StoreConfigInformation storeConfigInfo;
 
@@ -75,13 +92,19 @@ public class CompactionConfiguration {
   private final long maxCompactSize;
   private final long offPeakMaxCompactSize;
   private final long minCompactSize;
-  private final int minFilesToCompact;
+  /** This one can be update **/
+  private int minFilesToCompact;
   private final int maxFilesToCompact;
   private final double compactionRatio;
   private final long throttlePoint;
   private final long majorCompactionPeriod;
   private final float majorCompactionJitter;
   private final float minLocalityToForceCompact;
+  private final long maxStoreFileAgeMillis;
+  private final long baseWindowMillis;
+  private final int windowsPerTier;
+  private final int incomingWindowMin;
+  private final String compactionPolicyForTieredWindow;
 
   CompactionConfiguration(Configuration conf, StoreConfigInformation storeConfigInfo) {
     this.conf = conf;
@@ -104,6 +127,13 @@ public class CompactionConfiguration {
     // Make it 0.5 so jitter has us fall evenly either side of when the compaction should run
     majorCompactionJitter = conf.getFloat("hbase.hregion.majorcompaction.jitter", 0.50F);
     minLocalityToForceCompact = conf.getFloat(HBASE_HSTORE_MIN_LOCALITY_TO_SKIP_MAJOR_COMPACT, 0f);
+
+    maxStoreFileAgeMillis = conf.getLong(MAX_AGE_MILLIS_KEY, Long.MAX_VALUE);
+    baseWindowMillis = conf.getLong(BASE_WINDOW_MILLIS_KEY, 3600000 * 6);
+    windowsPerTier = conf.getInt(WINDOWS_PER_TIER_KEY, 4);
+    incomingWindowMin = conf.getInt(INCOMING_WINDOW_MIN_KEY, 6);
+    compactionPolicyForTieredWindow = conf.get(COMPACTION_POLICY_CLASS_FOR_TIERED_WINDOWS_KEY,
+        DEFAULT_TIER_COMPACTION_POLICY_CLASS.getName());
     LOG.info(this);
   }
 
@@ -111,7 +141,9 @@ public class CompactionConfiguration {
   public String toString() {
     return String.format(
       "size [%d, %d, %d); files [%d, %d); ratio %f; off-peak ratio %f; throttle point %d;"
-      + " major period %d, major jitter %f, min locality to compact %f",
+      + " major period %d, major jitter %f, min locality to compact %f;"
+      + " tiered compaction: max_age %d, base window in milliseconds %d, windows per tier %d,"
+      + "incoming window min %d",
       minCompactSize,
       maxCompactSize,
       offPeakMaxCompactSize,
@@ -122,7 +154,11 @@ public class CompactionConfiguration {
       throttlePoint,
       majorCompactionPeriod,
       majorCompactionJitter,
-      minLocalityToForceCompact);
+      minLocalityToForceCompact,
+      maxStoreFileAgeMillis,
+      baseWindowMillis,
+      windowsPerTier,
+      incomingWindowMin);
   }
 
   /**
@@ -147,6 +183,14 @@ public class CompactionConfiguration {
   }
 
   /**
+   * Set upper bound on number of files to be included in minor compactions
+   * @param threshold value to set to
+   */
+  public void setMinFilesToCompact(int threshold) {
+    minFilesToCompact = threshold;
+  }
+
+  /**
    * @return upper bound on number of files to be included in minor compactions
    */
   public int getMaxFilesToCompact() {
@@ -176,7 +220,7 @@ public class CompactionConfiguration {
 
   /**
    * @return Major compaction period from compaction.
-   * Major compactions are selected periodically according to this parameter plus jitter
+   *   Major compactions are selected periodically according to this parameter plus jitter
    */
   public long getMajorCompactionPeriod() {
     return majorCompactionPeriod;
@@ -184,7 +228,7 @@ public class CompactionConfiguration {
 
   /**
    * @return Major the jitter fraction, the fraction within which the major compaction
-   *  period is randomly chosen from the majorCompactionPeriod in each store.
+   *    period is randomly chosen from the majorCompactionPeriod in each store.
    */
   public float getMajorCompactionJitter() {
     return majorCompactionJitter;
@@ -192,8 +236,8 @@ public class CompactionConfiguration {
 
   /**
    * @return Block locality ratio, the ratio at which we will include old regions with a single
-   * store file for major compaction.  Used to improve block locality for regions that
-   * haven't had writes in a while but are still being read.
+   *   store file for major compaction.  Used to improve block locality for regions that
+   *   haven't had writes in a while but are still being read.
    */
   public float getMinLocalityToForceCompact() {
     return minLocalityToForceCompact;
@@ -202,7 +246,7 @@ public class CompactionConfiguration {
   public long getOffPeakMaxCompactSize() {
     return offPeakMaxCompactSize;
   }
-  
+
   public long getMaxCompactSize(boolean mayUseOffpeak) {
     if (mayUseOffpeak) {
       return getOffPeakMaxCompactSize();
@@ -210,4 +254,24 @@ public class CompactionConfiguration {
       return getMaxCompactSize();
     }
   }
+
+  public long getMaxStoreFileAgeMillis() {
+    return maxStoreFileAgeMillis;
+  }
+
+  public long getBaseWindowMillis() {
+    return baseWindowMillis;
+  }
+
+  public int getWindowsPerTier() {
+    return windowsPerTier;
+  }
+
+  public int getIncomingWindowMin() {
+    return incomingWindowMin;
+  }
+
+  public String getCompactionPolicyForTieredWindow() {
+    return compactionPolicyForTieredWindow;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f7f96b9f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
new file mode 100644
index 0000000..16b534c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
@@ -0,0 +1,294 @@
+/**
+ * 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.compactions;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.PeekingIterator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+
+/**
+ * HBASE-15181 This is a simple implementation of date-based tiered compaction similar to
+ * Cassandra's for the following benefits:
+ * 1. Improve date-range-based scan by structuring store files in date-based tiered layout.
+ * 2. Reduce compaction overhead.
+ * 3. Improve TTL efficiency.
+ * Perfect fit for the use cases that:
+ * 1. has mostly date-based data write and scan and a focus on the most recent data.
+ * 2. never or rarely deletes data. Out-of-order writes are handled gracefully. Time range
+ * overlapping among store files is tolerated and the performance impact is minimized. Configuration
+ * can be set at hbase-site or overriden at per-table or per-column-famly level by hbase shell.
+ * Design spec is at
+ * https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8/
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy {
+  private static final Log LOG = LogFactory.getLog(DateTieredCompactionPolicy.class);
+
+  private RatioBasedCompactionPolicy compactionPolicyPerWindow;
+
+  public DateTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo)
+      throws IOException {
+    super(conf, storeConfigInfo);
+    try {
+      compactionPolicyPerWindow =
+          ReflectionUtils.instantiateWithCustomCtor(comConf.getCompactionPolicyForTieredWindow(),
+            new Class[] { Configuration.class, StoreConfigInformation.class }, new Object[] { conf,
+              storeConfigInfo });
+    } catch (Exception e) {
+      throw new IOException("Unable to load configured compaction policy '"
+          + comConf.getCompactionPolicyForTieredWindow() + "'", e);
+    }
+  }
+
+  @Override
+  public boolean isMajorCompaction(Collection<StoreFile> filesToCompact) throws IOException {
+    // Never do major compaction unless forced
+    return false;
+  }
+
+  @Override
+  /**
+   * Heuristics for guessing whether we need compaction.
+   */
+  public boolean needsCompaction(final Collection<StoreFile> storeFiles,
+      final List<StoreFile> filesCompacting) {
+    return needsCompaction(storeFiles, filesCompacting, EnvironmentEdgeManager.currentTime());
+  }
+
+  @VisibleForTesting
+  public boolean needsCompaction(final Collection<StoreFile> storeFiles,
+      final List<StoreFile> filesCompacting, long now) {
+    if (!super.needsCompaction(storeFiles, filesCompacting)) {
+      return false;
+    }
+
+    ArrayList<StoreFile> candidates = new ArrayList<StoreFile>(storeFiles);
+    candidates = filterBulk(candidates);
+    candidates = skipLargeFiles(candidates, true);
+    try {
+      candidates = applyCompactionPolicy(candidates, true, false, now);
+    } catch (Exception e) {
+      LOG.error("Can not check for compaction: ", e);
+      return false;
+    }
+
+    return candidates != null && candidates.size() >= comConf.getMinFilesToCompact();
+  }
+
+  /**
+   * Could return null if no candidates are found
+   */
+  @Override
+  public ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
+      boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
+    return applyCompactionPolicy(candidates, mayUseOffPeak, mayBeStuck,
+      EnvironmentEdgeManager.currentTime());
+  }
+
+  /**
+   * Input candidates are sorted from oldest to newest by seqId. Could return null if no candidates
+   * are found.
+   */
+  @VisibleForTesting
+  public ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
+      boolean mayUseOffPeak, boolean mayBeStuck, long now) throws IOException {
+    Iterable<StoreFile> candidatesInWindow =
+      filterOldStoreFiles(Lists.newArrayList(candidates), comConf.getMaxStoreFileAgeMillis(), now);
+
+    List<ArrayList<StoreFile>> buckets =
+        partitionFilesToBuckets(candidatesInWindow, comConf.getBaseWindowMillis(),
+          comConf.getWindowsPerTier(), now);
+    LOG.debug("Compaction buckets are: " + buckets);
+
+    return newestBucket(buckets, comConf.getIncomingWindowMin(), now,
+      comConf.getBaseWindowMillis(), mayUseOffPeak);
+  }
+
+  /**
+   * @param buckets the list of buckets, sorted from newest to oldest, from which to return the
+   *          newest bucket within thresholds.
+   * @param incomingWindowThreshold minimum number of storeFiles in a bucket to qualify.
+   * @param maxThreshold maximum number of storeFiles to compact at once (the returned bucket will
+   *          be trimmed down to this).
+   * @return a bucket (a list of store files within a window to be compacted).
+   * @throws IOException error
+   */
+  private ArrayList<StoreFile> newestBucket(List<ArrayList<StoreFile>> buckets,
+      int incomingWindowThreshold, long now, long baseWindowMillis, boolean mayUseOffPeak)
+      throws IOException {
+    Window incomingWindow = getInitialWindow(now, baseWindowMillis);
+    for (ArrayList<StoreFile> bucket : buckets) {
+      int minThreshold =
+          incomingWindow.compareToTimestamp(bucket.get(0).getMaximumTimestamp()) <= 0 ? comConf
+              .getIncomingWindowMin() : comConf.getMinFilesToCompact();
+      compactionPolicyPerWindow.setMinThreshold(minThreshold);
+      ArrayList<StoreFile> candidates =
+          compactionPolicyPerWindow.applyCompactionPolicy(bucket, mayUseOffPeak, false);
+      if (candidates != null && !candidates.isEmpty()) {
+        return candidates;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * We receive store files sorted in ascending order by seqId then scan the list of files. If the
+   * current file has a maxTimestamp older than last known maximum, treat this file as it carries
+   * the last known maximum. This way both seqId and timestamp are in the same order. If files carry
+   * the same maxTimestamps, they are ordered by seqId. We then reverse the list so they are ordered
+   * by seqId and maxTimestamp in decending order and build the time windows. All the out-of-order
+   * data into the same compaction windows, guaranteeing contiguous compaction based on sequence id.
+   */
+  private static List<ArrayList<StoreFile>> partitionFilesToBuckets(Iterable<StoreFile> storeFiles,
+      long baseWindowSizeMillis, int windowsPerTier, long now) {
+    List<ArrayList<StoreFile>> buckets = Lists.newArrayList();
+    Window window = getInitialWindow(now, baseWindowSizeMillis);
+
+    List<Pair<StoreFile, Long>> storefileMaxTimestampPairs =
+        Lists.newArrayListWithCapacity(Iterables.size(storeFiles));
+    long maxTimestampSeen = Long.MIN_VALUE;
+    for (StoreFile storeFile : storeFiles) {
+      // if there is out-of-order data,
+      // we put them in the same window as the last file in increasing order
+      maxTimestampSeen = Math.max(maxTimestampSeen, storeFile.getMaximumTimestamp());
+      storefileMaxTimestampPairs.add(new Pair<StoreFile, Long>(storeFile, maxTimestampSeen));
+    }
+
+    Collections.reverse(storefileMaxTimestampPairs);
+    PeekingIterator<Pair<StoreFile, Long>> it =
+        Iterators.peekingIterator(storefileMaxTimestampPairs.iterator());
+
+    while (it.hasNext()) {
+      int compResult = window.compareToTimestamp(it.peek().getSecond());
+      if (compResult > 0) {
+        // If the file is too old for the window, switch to the next window
+        window = window.nextWindow(windowsPerTier);
+      } else {
+        // The file is within the target window
+        ArrayList<StoreFile> bucket = Lists.newArrayList();
+        // Add all files in the same window to current bucket. For incoming window
+        // we tolerate files with future data although it is sub-optimal
+        while (it.hasNext() && window.compareToTimestamp(it.peek().getSecond()) <= 0) {
+          bucket.add(it.next().getFirst());
+        }
+        if (!bucket.isEmpty()) {
+          buckets.add(bucket);
+        }
+      }
+    }
+
+    return buckets;
+  }
+
+  /**
+   * Removes all store files with max timestamp older than (current - maxAge).
+   * @param storeFiles all store files to consider
+   * @param maxAge the age in milliseconds when a store file stops participating in compaction.
+   * @param now current time. store files with max timestamp less than (now - maxAge) are filtered.
+   * @return a list of storeFiles with the store file older than maxAge excluded
+   */
+  private static Iterable<StoreFile> filterOldStoreFiles(List<StoreFile> storeFiles, long maxAge,
+      long now) {
+    if (maxAge == 0) {
+      return ImmutableList.of();
+    }
+    final long cutoff = now - maxAge;
+    return Iterables.filter(storeFiles, new Predicate<StoreFile>() {
+      @Override
+      public boolean apply(StoreFile storeFile) {
+        // Known findbugs issue to guava. SuppressWarning or Nonnull annotation don't work.
+        if (storeFile == null) {
+          throw new NullPointerException();
+        }
+        return storeFile.getMaximumTimestamp() >= cutoff;
+      }
+    });
+  }
+
+  private static Window getInitialWindow(long now, long timeUnit) {
+    return new Window(timeUnit, now / timeUnit);
+  }
+
+  /**
+   * This is the class we use to partition from epoch time to now into tiers of exponential sizes of
+   * windows.
+   */
+  private static final class Window {
+    /**
+     * How big a range of timestamps fit inside the window in milliseconds.
+     */
+    private final long windowMillis;
+
+    /**
+     * A timestamp t is within the window iff t / size == divPosition.
+     */
+    private final long divPosition;
+
+    private Window(long baseWindowMillis, long divPosition) {
+      this.windowMillis = baseWindowMillis;
+      this.divPosition = divPosition;
+    }
+
+    /**
+     * Compares the window to a timestamp.
+     * @param timestamp the timestamp to compare.
+     * @return a negative integer, zero, or a positive integer as the window lies before, covering,
+     *         or after than the timestamp.
+     */
+    public int compareToTimestamp(long timestamp) {
+      long pos = timestamp / windowMillis;
+      return divPosition == pos ? 0 : divPosition < pos ? -1 : 1;
+    }
+
+    /**
+     * Move to the new window of the same tier or of the next tier, which represents an earlier time
+     * span.
+     * @param windowsPerTier The number of contiguous windows that will have the same size. Windows
+     *          following those will be <code>tierBase</code> times as big.
+     * @return The next window
+     */
+    public Window nextWindow(int windowsPerTier) {
+      if (divPosition % windowsPerTier > 0) {
+        return new Window(windowMillis, divPosition - 1);
+      } else {
+        return new Window(windowMillis * windowsPerTier, divPosition / windowsPerTier - 1);
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f7f96b9f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
index a823d7c..4533a9c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
@@ -73,7 +73,9 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
   }
 
   /**
-   * @param candidateFiles candidate files, ordered from oldest to newest. All files in store.
+   * @param candidateFiles candidate files, ordered from oldest to newest by seqId. We rely on
+   *   DefaultStoreFileManager to sort the files by seqId to guarantee contiguous compaction based
+   *   on seqId for data consistency.
    * @return subset copy of candidate list that meets compaction criteria
    * @throws java.io.IOException
    */
@@ -128,7 +130,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
    * exclude all files above maxCompactSize
    * Also save all references. We MUST compact them
    */
-  private ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates, 
+  protected ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates,
     boolean mayUseOffpeak) {
     int pos = 0;
     while (pos < candidates.size() && !candidates.get(pos).isReference()
@@ -148,7 +150,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
    * @return filtered subset
    * exclude all bulk load files if configured
    */
-  private ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
+  protected ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
     candidates.removeAll(Collections2.filter(candidates,
         new Predicate<StoreFile>() {
           @Override
@@ -184,7 +186,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
    * @return filtered subset
    * forget the compactionSelection if we don't have enough files
    */
-  private ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates) {
+  protected ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates) {
     int minFiles = comConf.getMinFilesToCompact();
     if (candidates.size() < minFiles) {
       if(LOG.isDebugEnabled()) {
@@ -387,4 +389,12 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
     int numCandidates = storeFiles.size() - filesCompacting.size();
     return numCandidates >= comConf.getMinFilesToCompact();
   }
+
+  /**
+   * Overwrite min threshold for compaction
+   * @param minThreshold min to update to
+   */
+  public void setMinThreshold(int minThreshold) {
+    comConf.setMinFilesToCompact(minThreshold);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f7f96b9f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
index 3614846..df039e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
@@ -100,6 +100,18 @@ public class MockStoreFile extends StoreFile {
     this.entryCount = entryCount;
   }
 
+  public Long getMinimumTimestamp() {
+    return (timeRangeTracker == null) ?
+        null :
+        timeRangeTracker.getMinimumTimestamp();
+  }
+
+  public Long getMaximumTimestamp() {
+    return (timeRangeTracker == null) ?
+        null :
+        timeRangeTracker.getMaximumTimestamp();
+  }
+
   @Override
   public StoreFile.Reader getReader() {
     final long len = this.length;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f7f96b9f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java
new file mode 100644
index 0000000..cfb54b7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java
@@ -0,0 +1,211 @@
+/**
+ *
+ * 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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestDateTieredCompaction extends TestCompactionPolicy {
+  ArrayList<StoreFile> sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes)
+      throws IOException {
+    ArrayList<Long> ageInDisk = new ArrayList<Long>();
+    for (int i = 0; i < sizes.length; i++) {
+      ageInDisk.add(0L);
+    }
+
+    ArrayList<StoreFile> ret = Lists.newArrayList();
+    for (int i = 0; i < sizes.length; i++) {
+      MockStoreFile msf =
+          new MockStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i);
+      msf.setTimeRangeTracker(new TimeRangeTracker(minTimestamps[i], maxTimestamps[i]));
+      ret.add(msf);
+    }
+    return ret;
+  }
+
+  @Override
+  protected void config() {
+    super.config();
+
+    // Set up policy
+    conf.setLong(CompactionConfiguration.MAX_AGE_MILLIS_KEY, 100);
+    conf.setLong(CompactionConfiguration.INCOMING_WINDOW_MIN_KEY, 3);
+    conf.setLong(CompactionConfiguration.BASE_WINDOW_MILLIS_KEY, 6);
+    conf.setInt(CompactionConfiguration.WINDOWS_PER_TIER_KEY, 4);
+    conf.set(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
+      DateTieredCompactionPolicy.class.getName());
+
+    // Special settings for compaction policy per window
+    this.conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 2);
+    this.conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 12);
+    this.conf.setFloat(CompactionConfiguration.HBASE_HSTORE_COMPACTION_RATIO_KEY, 1.2F);
+  }
+
+  void compactEquals(long now, ArrayList<StoreFile> candidates, long... expected)
+      throws IOException {
+    Assert.assertTrue(((DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy())
+        .needsCompaction(candidates, ImmutableList.<StoreFile> of(), now));
+
+    List<StoreFile> actual =
+        ((DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy())
+            .applyCompactionPolicy(candidates, false, false, now);
+
+    Assert.assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
+  }
+
+  /**
+   * Test for incoming window
+   * @throws IOException with error
+   */
+  @Test
+  public void incomingWindow() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+    long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11, 12, 13 };
+
+    compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 13, 12, 11, 10);
+  }
+
+  /**
+   * Not enough files in incoming window
+   * @throws IOException with error
+   */
+  @Test
+  public void NotIncomingWindow() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 };
+    long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11 };
+
+    compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 25, 24, 23, 22, 21, 20);
+  }
+
+  /**
+   * Test for file newer than incoming window
+   * @throws IOException with error
+   */
+  @Test
+  public void NewerThanIncomingWindow() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 18 };
+    long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11, 12, 13 };
+
+    compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 13, 12, 11, 10);
+  }
+
+  /**
+   * If there is no T1 window, we don't build 2
+   * @throws IOException with error
+   */
+  @Test
+  public void NoT2() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 44, 60, 61, 92, 95, 100 };
+    long[] sizes = new long[] { 0, 20, 21, 22, 23, 1 };
+
+    compactEquals(100, sfCreate(minTimestamps, maxTimestamps, sizes), 23, 22);
+  }
+
+  @Test
+  public void T1() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 120, 124, 143, 145, 157 };
+    long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 30, 31, 32, 2, 1 };
+
+    compactEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), 32, 31, 30);
+  }
+
+  /**
+   * Apply exploring logic on non-incoming window
+   * @throws IOException with error
+   */
+  @Test
+  public void RatioT0() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 };
+    long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 280, 23, 24, 1 };
+
+    compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 22, 21, 20);
+  }
+
+  /**
+   * Also apply ratio-based logic on t2 window
+   * @throws IOException with error
+   */
+  @Test
+  public void RatioT2() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 120, 124, 143, 145, 157 };
+    long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 350, 30, 31, 2, 1 };
+
+    compactEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), 31, 30);
+  }
+
+  /**
+   * The next compaction call after testTieredCompactionRatioT0 is compacted
+   * @throws IOException with error
+   */
+  @Test
+  public void RatioT0Next() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 8, 9, 10, 11, 12 };
+    long[] sizes = new long[] { 30, 31, 32, 33, 34, 22, 280, 23, 24, 1 };
+
+    compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 24, 23);
+  }
+
+  /**
+   * Older than now(161) - maxAge(100)
+   * @throws IOException with error
+   */
+  @Test
+  public void olderThanMaxAge() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 105, 106, 113, 145, 157 };
+    long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 33, 30, 31, 2, 1 };
+
+    compactEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), 31, 30, 33, 42, 41, 40);
+  }
+
+  /**
+   * Out-of-order data
+   * @throws IOException with error
+   */
+  @Test
+  public void OutOfOrder() throws IOException {
+    long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+    long[] maxTimestamps = new long[] { 0, 13, 3, 10, 11, 1, 2, 12, 14, 15 };
+    long[] sizes = new long[] { 30, 31, 32, 33, 34, 22, 28, 23, 24, 1 };
+
+    compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 1, 24, 23, 28, 22, 34, 33, 32,
+      31);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f7f96b9f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
index d68c6b1..dbd6f11 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
@@ -19,188 +19,20 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
-import junit.framework.TestCase;
-
-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.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
-import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.junit.After;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.google.common.collect.Lists;
-
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestDefaultCompactSelection extends TestCase {
-  private final static Log LOG = LogFactory.getLog(TestDefaultCompactSelection.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  protected Configuration conf;
-  protected HStore store;
-  private static final String DIR=
-    TEST_UTIL.getDataTestDir(TestDefaultCompactSelection.class.getSimpleName()).toString();
-  private static Path TEST_FILE;
-
-  protected static final int minFiles = 3;
-  protected static final int maxFiles = 5;
-
-  protected static final long minSize = 10;
-  protected static final long maxSize = 2100;
-
-  private WALFactory wals;
-  private HRegion region;
-
-  @Override
-  public void setUp() throws Exception {
-    // setup config values necessary for store
-    this.conf = TEST_UTIL.getConfiguration();
-    this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
-    this.conf.setInt("hbase.hstore.compaction.min", minFiles);
-    this.conf.setInt("hbase.hstore.compaction.max", maxFiles);
-    this.conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, minSize);
-    this.conf.setLong("hbase.hstore.compaction.max.size", maxSize);
-    this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F);
-    // Test depends on this not being set to pass.  Default breaks test.  TODO: Revisit.
-    this.conf.unset("hbase.hstore.compaction.min.size");
-
-    //Setting up a Store
-    final String id = TestDefaultCompactSelection.class.getName();
-    Path basedir = new Path(DIR);
-    final Path logdir = new Path(basedir, DefaultWALProvider.getWALDirectoryName(id));
-    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family"));
-    FileSystem fs = FileSystem.get(conf);
-
-    fs.delete(logdir, true);
-
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("table")));
-    htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-
-    final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, basedir);
-    wals = new WALFactory(walConf, null, id);
-    region = HBaseTestingUtility.createRegionAndWAL(info, basedir, conf, htd);
-    HBaseTestingUtility.closeRegionAndWAL(region);
-    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
-    region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable()
-        .getNamespace()), fs, conf, info, htd, null);
-
-    store = new HStore(region, hcd, conf);
-
-    TEST_FILE = region.getRegionFileSystem().createTempName();
-    fs.createNewFile(TEST_FILE);
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    IOException ex = null;
-    try {
-      region.close();
-    } catch (IOException e) {
-      LOG.warn("Caught Exception", e);
-      ex = e;
-    }
-    try {
-      wals.close();
-    } catch (IOException e) {
-      LOG.warn("Caught Exception", e);
-      ex = e;
-    }
-    if (ex != null) {
-      throw ex;
-    }
-  }
-
-  ArrayList<Long> toArrayList(long... numbers) {
-    ArrayList<Long> result = new ArrayList<Long>();
-    for (long i : numbers) {
-      result.add(i);
-    }
-    return result;
-  }
-
-  List<StoreFile> sfCreate(long... sizes) throws IOException {
-    ArrayList<Long> ageInDisk = new ArrayList<Long>();
-    for (int i = 0; i < sizes.length; i++) {
-      ageInDisk.add(0L);
-    }
-    return sfCreate(toArrayList(sizes), ageInDisk);
-  }
-
-  List<StoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
-    throws IOException {
-    return sfCreate(false, sizes, ageInDisk);
-  }
-
-  List<StoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
-    ArrayList<Long> ageInDisk = new ArrayList<Long>(sizes.length);
-    for (int i = 0; i < sizes.length; i++) {
-      ageInDisk.add(0L);
-    }
-    return sfCreate(isReference, toArrayList(sizes), ageInDisk);
-  }
-
-  List<StoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
-      throws IOException {
-    List<StoreFile> ret = Lists.newArrayList();
-    for (int i = 0; i < sizes.size(); i++) {
-      ret.add(new MockStoreFile(TEST_UTIL, TEST_FILE,
-          sizes.get(i), ageInDisk.get(i), isReference, i));
-    }
-    return ret;
-  }
-
-  long[] getSizes(List<StoreFile> sfList) {
-    long[] aNums = new long[sfList.size()];
-    for (int i = 0; i < sfList.size(); ++i) {
-      aNums[i] = sfList.get(i).getReader().length();
-    }
-    return aNums;
-  }
-
-  void compactEquals(List<StoreFile> candidates, long... expected)
-    throws IOException {
-    compactEquals(candidates, false, false, expected);
-  }
-
-  void compactEquals(List<StoreFile> candidates, boolean forcemajor, long... expected)
-    throws IOException {
-    compactEquals(candidates, forcemajor, false, expected);
-  }
-
-  void compactEquals(List<StoreFile> candidates, boolean forcemajor, boolean isOffPeak,
-      long ... expected)
-  throws IOException {
-    store.forceMajor = forcemajor;
-    //Test Default compactions
-    CompactionRequest result = ((RatioBasedCompactionPolicy)store.storeEngine.getCompactionPolicy())
-        .selectCompaction(candidates, new ArrayList<StoreFile>(), false, isOffPeak, forcemajor);
-    List<StoreFile> actual = new ArrayList<StoreFile>(result.getFiles());
-    if (isOffPeak && !forcemajor) {
-      assertTrue(result.isOffPeak());
-    }
-    assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
-    store.forceMajor = false;
-  }
+@Category(SmallTests.class)
+public class TestDefaultCompactSelection extends TestCompactionPolicy {
 
+  @Test
   public void testCompactionRatio() throws IOException {
     /**
      * NOTE: these tests are specific to describe the implementation of the
@@ -278,9 +110,10 @@ public class TestDefaultCompactSelection extends TestCase {
     // empty case
     compactEquals(new ArrayList<StoreFile>() /* empty */);
     // empty case (because all files are too big)
-   compactEquals(sfCreate(tooBig, tooBig) /* empty */);
+    compactEquals(sfCreate(tooBig, tooBig) /* empty */);
   }
 
+  @Test
   public void testOffPeakCompactionRatio() throws IOException {
     /*
      * NOTE: these tests are specific to describe the implementation of the
@@ -295,6 +128,7 @@ public class TestDefaultCompactSelection extends TestCase {
     compactEquals(sfCreate(999, 50, 12, 12, 1), 12, 12, 1);
   }
 
+  @Test
   public void testStuckStoreCompaction() throws IOException {
     // Select the smallest compaction if the store is stuck.
     compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,30), 30, 30, 30);
@@ -309,6 +143,7 @@ public class TestDefaultCompactSelection extends TestCase {
     compactEquals(sfCreate(99,99,99,99, 27,27,27,20,20,20), 20, 20, 20);
   }
 
+  @Test
   public void testCompactionEmptyHFile() throws IOException {
     // Set TTL
     ScanInfo oldScanInfo = store.getScanInfo();
@@ -330,7 +165,7 @@ public class TestDefaultCompactSelection extends TestCase {
     CompactionRequest result = ((RatioBasedCompactionPolicy) store.storeEngine
         .getCompactionPolicy()).selectCompaction(candidates,
         new ArrayList<StoreFile>(), false, false, false);
-    assertTrue(result.getFiles().size() == 0);
+    Assert.assertTrue(result.getFiles().size() == 0);
     store.setScanInfo(oldScanInfo);
   }
 }


[19/50] [abbrv] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by en...@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>

[12/50] [abbrv] hbase git commit: HBASE-15319 clearJmxCache does not take effect actually

Posted by en...@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-7912
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;
     }


[41/50] [abbrv] hbase git commit: HBASE-14878 Add hbase-shaded-client archetype to hbase-archetypes

Posted by en...@apache.org.
HBASE-14878 Add hbase-shaded-client archetype to hbase-archetypes

Signed-off-by: Elliott Clark <ec...@apache.org>


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

Branch: refs/heads/HBASE-7912
Commit: 83297f661b80af58190591c57d3cef1e6496e56b
Parents: 88f7759
Author: Daniel Vimont <da...@commonvox.org>
Authored: Tue Feb 23 13:13:56 2016 +0900
Committer: Elliott Clark <ec...@apache.org>
Committed: Mon Feb 29 09:09:37 2016 -0800

----------------------------------------------------------------------
 hbase-archetypes/README.md                      |  10 +-
 .../hbase-archetype-builder/createArchetypes.sh |   4 +
 .../installArchetypes.sh                        |   4 +
 .../hbase-archetype-builder/pom.xml             |  74 +++++-
 .../hbase-shaded-client-project/pom.xml         |  76 +++++++
 .../exemplars/shaded_client/HelloHBase.java     | 226 +++++++++++++++++++
 .../exemplars/shaded_client/package-info.java   |  25 ++
 .../src/main/resources/log4j.properties         | 111 +++++++++
 .../exemplars/shaded_client/TestHelloHBase.java | 131 +++++++++++
 hbase-archetypes/pom.xml                        |   3 +-
 10 files changed, 659 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/README.md
----------------------------------------------------------------------
diff --git a/hbase-archetypes/README.md b/hbase-archetypes/README.md
index 3af1f8b..7997b56 100644
--- a/hbase-archetypes/README.md
+++ b/hbase-archetypes/README.md
@@ -81,11 +81,15 @@ of the new archetype. (It may be most straightforward to simply copy the `src`
 and `pom.xml` components from one of the existing exemplar projects, replace
 the `src/main` and `src/test` code, and modify the `pom.xml` file's
 `<dependencies>`, `<artifactId>`,` <name>`, and `<description>` elements.)
-2. Modify the `hbase-archetype-builder/pom.xml` file: (a) add the new exemplar
-project to the `<modules>` element, and (b) add appropriate `<execution>`
+2. Modify the `hbase-archetypes/pom.xml` file: add a new `<module>` subelement
+to the `<modules>` element, with the new exemplar project's subdirectory name
+as its value.
+3. Modify the `hbase-archetype-builder/pom.xml` file: (a) add a new `<*.dir>`
+subelement to the `<properties>` element, with the new exemplar project's
+subdirectory name as its value, and (b) add appropriate `<execution>`
 elements and `<transformationSet>` elements within the `<plugin>` elements
 (using the existing entries from already-existing exemplar projects as a guide).
-3. Add appropriate entries for the new exemplar project to the
+4. Add appropriate entries for the new exemplar project to the
 `createArchetypes.sh` and `installArchetypes.sh` scripts in the
 `hbase-archetype-builder` subdirectory (using the existing entries as a guide).
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-archetype-builder/createArchetypes.sh
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-archetype-builder/createArchetypes.sh b/hbase-archetypes/hbase-archetype-builder/createArchetypes.sh
index 3aeb1c3..067fbd9 100755
--- a/hbase-archetypes/hbase-archetype-builder/createArchetypes.sh
+++ b/hbase-archetypes/hbase-archetype-builder/createArchetypes.sh
@@ -25,6 +25,10 @@ buildArchetypeSubdir=target/build-archetype
 cd /"$workingDir"/../hbase-client-project/$buildArchetypeSubdir
 mvn archetype:create-from-project
 
+# CREATE hbase-shaded-client archetype
+cd /"$workingDir"/../hbase-shaded-client-project/$buildArchetypeSubdir
+mvn archetype:create-from-project
+
 # add entries for additional archetypes above this comment (modeled on entries above)
 
 cd "$workingDir"

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-archetype-builder/installArchetypes.sh
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-archetype-builder/installArchetypes.sh b/hbase-archetypes/hbase-archetype-builder/installArchetypes.sh
index 74f118e..1067a1f 100755
--- a/hbase-archetypes/hbase-archetype-builder/installArchetypes.sh
+++ b/hbase-archetypes/hbase-archetype-builder/installArchetypes.sh
@@ -26,6 +26,10 @@ archetypeSourceSubdir=target/generated-sources/archetype
 cd /"$workingDir"/../hbase-client-project/$buildArchetypeSubdir/$archetypeSourceSubdir
 mvn install
 
+# INSTALL hbase-shaded-client archetype
+cd /"$workingDir"/../hbase-shaded-client-project/$buildArchetypeSubdir/$archetypeSourceSubdir
+mvn install
+
 # add entries for additional archetypes above this comment (modeled on entries above)
 
 cd "$workingDir"

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-archetype-builder/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-archetype-builder/pom.xml b/hbase-archetypes/hbase-archetype-builder/pom.xml
index 00a1305..1672dea 100644
--- a/hbase-archetypes/hbase-archetype-builder/pom.xml
+++ b/hbase-archetypes/hbase-archetype-builder/pom.xml
@@ -41,6 +41,7 @@
     <temp.exemplar.subdir>target/temp</temp.exemplar.subdir>
     <temp.archetype.subdir>target/temp-arch</temp.archetype.subdir>
     <hbase-client.dir>hbase-client-project</hbase-client.dir>
+    <hbase-shaded-client.dir>hbase-shaded-client-project</hbase-shaded-client.dir>
     <!-- For new archetype, add corresponding *.dir property above this comment.
          (See hbase-archetypes/README.txt for details on adding new archetype.) -->
   </properties>
@@ -52,7 +53,7 @@
         <version>2.7</version>
         <executions>
           <!-- maven-resources-plugin copies each exemplar project's src directory to
-            ${build.archetype.subdir} subdirectory, and copy each project's pom.xml file
+            ${build.archetype.subdir} subdirectory, and copies each project's pom.xml file
             to ${temp.exemplar.subdir} subdirectory. (Filtering during copy replaces
             ${project-version} with literal value). The pom.xml files are processed
             further using xml-maven-plugin for xslt transformation, below. -->
@@ -93,6 +94,43 @@
               </resources>
             </configuration>
           </execution>
+          <execution>
+            <id>hbase-shaded-client__copy-src-to-build-archetype-subdir</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>copy-resources</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>/${project.basedir}/../${hbase-shaded-client.dir}/${build.archetype.subdir}</outputDirectory>
+              <resources>
+                <resource>
+                  <directory>/${project.basedir}/../${hbase-shaded-client.dir}</directory>
+                  <includes>
+                    <include>src/**</include>
+                  </includes>
+                </resource>
+              </resources>
+            </configuration>
+          </execution>
+          <execution>
+            <id>hbase-shaded-client__copy-pom-to-temp-for-xslt-processing</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>copy-resources</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>/${project.basedir}/../${hbase-shaded-client.dir}/${temp.exemplar.subdir}</outputDirectory>
+              <resources>
+                <resource>
+                  <directory>/${project.basedir}/../${hbase-shaded-client.dir}</directory>
+                  <filtering>true</filtering> <!-- filtering replaces ${project.version} with literal -->
+                  <includes>
+                    <include>pom.xml</include>
+                  </includes>
+                 </resource>
+              </resources>
+            </configuration>
+          </execution>
           <!-- For new archetype, add pair of <execution> elements (modeled on existing elements) above this comment. -->
 
           <!-- maven-resources-plugin copies each archetype project's pom.xml file
@@ -116,6 +154,24 @@
               </resources>
             </configuration>
           </execution>
+          <execution>
+            <id>hbase-shaded-client-ARCHETYPE__copy-pom-to-temp-for-xslt-processing</id>
+            <phase>prepare-package</phase>
+            <goals>
+              <goal>copy-resources</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>/${project.basedir}/../${hbase-shaded-client.dir}/${temp.archetype.subdir}</outputDirectory>
+              <resources>
+                <resource>
+                  <directory>/${project.basedir}/../${hbase-shaded-client.dir}/${build.archetype.subdir}/${archetype.source.subdir}</directory>
+                  <includes>
+                    <include>pom.xml</include>
+                  </includes>
+                 </resource>
+              </resources>
+            </configuration>
+          </execution>
           <!-- For new archetype, add <execution> element (modeled on existing elements) above this comment. -->
         </executions>
       </plugin>
@@ -142,6 +198,14 @@
                   <outputDir>/${project.basedir}/../${hbase-client.dir}/${build.archetype.subdir}</outputDir>
                   <stylesheet>modify_exemplar_pom.xsl</stylesheet>
                 </transformationSet>
+                <transformationSet>
+                  <dir>/${project.basedir}/../${hbase-shaded-client.dir}/${temp.exemplar.subdir}</dir>
+                  <includes>
+                    <include>pom.xml</include>
+                  </includes>
+                  <outputDir>/${project.basedir}/../${hbase-shaded-client.dir}/${build.archetype.subdir}</outputDir>
+                  <stylesheet>modify_exemplar_pom.xsl</stylesheet>
+                </transformationSet>
                 <!-- For new archetype, add <transformationSet> element (modeled on existing elements) above this comment. -->
               </transformationSets>
             </configuration>
@@ -164,6 +228,14 @@
                   <outputDir>/${project.basedir}/../${hbase-client.dir}/${build.archetype.subdir}/${archetype.source.subdir}</outputDir>
                   <stylesheet>modify_archetype_pom.xsl</stylesheet>
                 </transformationSet>
+                <transformationSet>
+                  <dir>/${project.basedir}/../${hbase-shaded-client.dir}/${temp.archetype.subdir}</dir>
+                  <includes>
+                    <include>pom.xml</include>
+                  </includes>
+                  <outputDir>/${project.basedir}/../${hbase-shaded-client.dir}/${build.archetype.subdir}/${archetype.source.subdir}</outputDir>
+                  <stylesheet>modify_archetype_pom.xsl</stylesheet>
+                </transformationSet>
                 <!-- For new archetype, add <transformationSet> element (modeled on existing elements) above this comment. -->
               </transformationSets>
             </configuration>

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-shaded-client-project/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/pom.xml b/hbase-archetypes/hbase-shaded-client-project/pom.xml
new file mode 100644
index 0000000..6f73033
--- /dev/null
+++ b/hbase-archetypes/hbase-shaded-client-project/pom.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation=
+           "http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+/**
+ * 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.
+ */
+-->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase-archetypes</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>hbase-shaded-client-project</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache HBase - Exemplar for hbase-shaded-client archetype</name>
+  <description>Exemplar project for archetype with hbase-shaded-client dependency</description>
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <maven.compiler.source>${compileSource}</maven.compiler.source>
+    <maven.compiler.target>${compileSource}</maven.compiler.target>
+    <surefire.version>2.19</surefire.version>
+    <junit.version>4.12</junit.version>
+  </properties>
+
+  <build>
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-surefire-plugin</artifactId>
+          <version>${surefire.version}</version>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+
+  <dependencies>
+    <!-- Dependency for hbase-testing-util must precede compile-scoped dependencies. -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-shaded-client</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
new file mode 100644
index 0000000..1c6b8bb
--- /dev/null
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
@@ -0,0 +1,226 @@
+/**
+ *
+ * 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.hbase.archetypes.exemplars.shaded_client;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Successful running of this application requires access to an active instance
+ * of HBase. For install instructions for a standalone instance of HBase, please
+ * refer to https://hbase.apache.org/book.html#quickstart
+ */
+public final class HelloHBase {
+
+  protected static final String MY_NAMESPACE_NAME = "myTestNamespace";
+  static final TableName MY_TABLE_NAME = TableName.valueOf("myTestTable");
+  static final byte[] MY_COLUMN_FAMILY_NAME = Bytes.toBytes("cf");
+  static final byte[] MY_FIRST_COLUMN_QUALIFIER
+          = Bytes.toBytes("myFirstColumn");
+  static final byte[] MY_SECOND_COLUMN_QUALIFIER
+          = Bytes.toBytes("mySecondColumn");
+  static final byte[] MY_ROW_ID = Bytes.toBytes("rowId01");
+
+  // Private constructor included here to avoid checkstyle warnings
+  private HelloHBase() {
+  }
+
+  public static void main(final String[] args) throws IOException {
+    final boolean deleteAllAtEOJ = true;
+
+    /**
+     * ConnectionFactory#createConnection() automatically looks for
+     * hbase-site.xml (HBase configuration parameters) on the system's
+     * CLASSPATH, to enable creation of Connection to HBase via Zookeeper.
+     */
+    try (Connection connection = ConnectionFactory.createConnection();
+            Admin admin = connection.getAdmin()) {
+
+      admin.getClusterStatus(); // assure connection successfully established
+      System.out.println("\n*** Hello HBase! -- Connection has been "
+              + "established via Zookeeper!!\n");
+
+      createNamespaceAndTable(admin);
+
+      System.out.println("Getting a Table object for [" + MY_TABLE_NAME
+              + "] with which to perform CRUD operations in HBase.");
+      try (Table table = connection.getTable(MY_TABLE_NAME)) {
+
+        putRowToTable(table);
+        getAndPrintRowContents(table);
+
+        if (deleteAllAtEOJ) {
+          deleteRow(table);
+        }
+      }
+
+      if (deleteAllAtEOJ) {
+        deleteNamespaceAndTable(admin);
+      }
+    }
+  }
+
+  /**
+   * Invokes Admin#createNamespace and Admin#createTable to create a namespace
+   * with a table that has one column-family.
+   *
+   * @param admin Standard Admin object
+   * @throws IOException If IO problem encountered
+   */
+  static void createNamespaceAndTable(final Admin admin) throws IOException {
+
+    if (!namespaceExists(admin, MY_NAMESPACE_NAME)) {
+      System.out.println("Creating Namespace [" + MY_NAMESPACE_NAME + "].");
+
+      admin.createNamespace(NamespaceDescriptor
+              .create(MY_NAMESPACE_NAME).build());
+    }
+    if (!admin.tableExists(MY_TABLE_NAME)) {
+      System.out.println("Creating Table [" + MY_TABLE_NAME.getNameAsString()
+              + "], with one Column Family ["
+              + Bytes.toString(MY_COLUMN_FAMILY_NAME) + "].");
+
+      admin.createTable(new HTableDescriptor(MY_TABLE_NAME)
+              .addFamily(new HColumnDescriptor(MY_COLUMN_FAMILY_NAME)));
+    }
+  }
+
+  /**
+   * Invokes Table#put to store a row (with two new columns created 'on the
+   * fly') into the table.
+   *
+   * @param table Standard Table object (used for CRUD operations).
+   * @throws IOException If IO problem encountered
+   */
+  static void putRowToTable(final Table table) throws IOException {
+
+    table.put(new Put(MY_ROW_ID).addColumn(MY_COLUMN_FAMILY_NAME,
+            MY_FIRST_COLUMN_QUALIFIER,
+            Bytes.toBytes("Hello")).addColumn(MY_COLUMN_FAMILY_NAME,
+                    MY_SECOND_COLUMN_QUALIFIER,
+                    Bytes.toBytes("World!")));
+
+    System.out.println("Row [" + Bytes.toString(MY_ROW_ID)
+            + "] was put into Table ["
+            + table.getName().getNameAsString() + "] in HBase;\n"
+            + "  the row's two columns (created 'on the fly') are: ["
+            + Bytes.toString(MY_COLUMN_FAMILY_NAME) + ":"
+            + Bytes.toString(MY_FIRST_COLUMN_QUALIFIER)
+            + "] and [" + Bytes.toString(MY_COLUMN_FAMILY_NAME) + ":"
+            + Bytes.toString(MY_SECOND_COLUMN_QUALIFIER) + "]");
+  }
+
+  /**
+   * Invokes Table#get and prints out the contents of the retrieved row.
+   *
+   * @param table Standard Table object
+   * @throws IOException If IO problem encountered
+   */
+  static void getAndPrintRowContents(final Table table) throws IOException {
+
+    Result row = table.get(new Get(MY_ROW_ID));
+
+    System.out.println("Row [" + Bytes.toString(row.getRow())
+            + "] was retrieved from Table ["
+            + table.getName().getNameAsString()
+            + "] in HBase, with the following content:");
+
+    for (Entry<byte[], NavigableMap<byte[], byte[]>> colFamilyEntry
+            : row.getNoVersionMap().entrySet()) {
+      String columnFamilyName = Bytes.toString(colFamilyEntry.getKey());
+
+      System.out.println("  Columns in Column Family [" + columnFamilyName
+              + "]:");
+
+      for (Entry<byte[], byte[]> columnNameAndValueMap
+              : colFamilyEntry.getValue().entrySet()) {
+
+        System.out.println("    Value of Column [" + columnFamilyName + ":"
+                + Bytes.toString(columnNameAndValueMap.getKey()) + "] == "
+                + Bytes.toString(columnNameAndValueMap.getValue()));
+      }
+    }
+  }
+
+  /**
+   * Checks to see whether a namespace exists.
+   *
+   * @param admin Standard Admin object
+   * @param namespaceName Name of namespace
+   * @return true If namespace exists
+   * @throws IOException If IO problem encountered
+   */
+  static boolean namespaceExists(final Admin admin, final String namespaceName)
+          throws IOException {
+    try {
+      admin.getNamespaceDescriptor(namespaceName);
+    } catch (NamespaceNotFoundException e) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Invokes Table#delete to delete test data (i.e. the row)
+   *
+   * @param table Standard Table object
+   * @throws IOException If IO problem is encountered
+   */
+  static void deleteRow(final Table table) throws IOException {
+    System.out.println("Deleting row [" + Bytes.toString(MY_ROW_ID)
+            + "] from Table ["
+            + table.getName().getNameAsString() + "].");
+    table.delete(new Delete(MY_ROW_ID));
+  }
+
+  /**
+   * Invokes Admin#disableTable, Admin#deleteTable, and Admin#deleteNamespace to
+   * disable/delete Table and delete Namespace.
+   *
+   * @param admin Standard Admin object
+   * @throws IOException If IO problem is encountered
+   */
+  static void deleteNamespaceAndTable(final Admin admin) throws IOException {
+    if (admin.tableExists(MY_TABLE_NAME)) {
+      System.out.println("Disabling/deleting Table ["
+              + MY_TABLE_NAME.getNameAsString() + "].");
+      admin.disableTable(MY_TABLE_NAME); // Disable a table before deleting it.
+      admin.deleteTable(MY_TABLE_NAME);
+    }
+    if (namespaceExists(admin, MY_NAMESPACE_NAME)) {
+      System.out.println("Deleting Namespace [" + MY_NAMESPACE_NAME + "].");
+      admin.deleteNamespace(MY_NAMESPACE_NAME);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/package-info.java b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/package-info.java
new file mode 100644
index 0000000..754be16
--- /dev/null
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/package-info.java
@@ -0,0 +1,25 @@
+/*
+ *
+ * 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.
+ */
+
+/**
+ * This package provides fully-functional exemplar Java code demonstrating
+ * simple usage of the hbase-client API, for incorporation into a Maven
+ * archetype with hbase-shaded-client dependency.
+ */
+package org.apache.hbase.archetypes.exemplars.shaded_client;

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties b/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
new file mode 100644
index 0000000..d7c4552
--- /dev/null
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/resources/log4j.properties
@@ -0,0 +1,111 @@
+# 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.
+
+# Define some default values that can be overridden by system properties
+hbase.root.logger=INFO,console
+hbase.security.logger=INFO,console
+hbase.log.dir=.
+hbase.log.file=hbase.log
+
+# Define the root logger to the system property "hbase.root.logger".
+log4j.rootLogger=${hbase.root.logger}
+
+# Logging Threshold
+log4j.threshold=ALL
+
+#
+# Daily Rolling File Appender
+#
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hbase.log.dir}/${hbase.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+
+# Rolling File Appender properties
+hbase.log.maxfilesize=256MB
+hbase.log.maxbackupindex=20
+
+# Rolling File Appender
+log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+log4j.appender.RFA.File=${hbase.log.dir}/${hbase.log.file}
+
+log4j.appender.RFA.MaxFileSize=${hbase.log.maxfilesize}
+log4j.appender.RFA.MaxBackupIndex=${hbase.log.maxbackupindex}
+
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+
+#
+# Security audit appender
+#
+hbase.security.log.file=SecurityAuth.audit
+hbase.security.log.maxfilesize=256MB
+hbase.security.log.maxbackupindex=20
+log4j.appender.RFAS=org.apache.log4j.RollingFileAppender
+log4j.appender.RFAS.File=${hbase.log.dir}/${hbase.security.log.file}
+log4j.appender.RFAS.MaxFileSize=${hbase.security.log.maxfilesize}
+log4j.appender.RFAS.MaxBackupIndex=${hbase.security.log.maxbackupindex}
+log4j.appender.RFAS.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.category.SecurityLogger=${hbase.security.logger}
+log4j.additivity.SecurityLogger=false
+#log4j.logger.SecurityLogger.org.apache.hadoop.hbase.security.access.AccessController=TRACE
+#log4j.logger.SecurityLogger.org.apache.hadoop.hbase.security.visibility.VisibilityController=TRACE
+
+#
+# Null Appender
+#
+log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+
+# Custom Logging levels
+
+log4j.logger.org.apache.zookeeper=INFO
+#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+log4j.logger.org.apache.hadoop.hbase=INFO
+# Make these two classes INFO-level. Make them DEBUG to see more zk debug.
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+#log4j.logger.org.apache.hadoop.dfs=DEBUG
+# Set this class to log INFO only otherwise its OTT
+# Enable this to get detailed connection error/retry logging.
+# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+
+
+# Uncomment this line to enable tracing on _every_ RPC call (this can be a lot of output)
+#log4j.logger.org.apache.hadoop.ipc.HBaseServer.trace=DEBUG
+
+# Uncomment the below if you want to remove logging of client region caching'
+# and scan of hbase:meta messages
+# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=INFO
+# log4j.logger.org.apache.hadoop.hbase.client.MetaScanner=INFO

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java
new file mode 100644
index 0000000..cc5e201
--- /dev/null
+++ b/hbase-archetypes/hbase-shaded-client-project/src/test/java/org/apache/hbase/archetypes/exemplars/shaded_client/TestHelloHBase.java
@@ -0,0 +1,131 @@
+/**
+ *
+ * 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.hbase.archetypes.exemplars.shaded_client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import static org.junit.Assert.assertEquals;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Unit testing for HelloHBase.
+ */
+@Category(MediumTests.class)
+public class TestHelloHBase {
+
+  private static final HBaseTestingUtility TEST_UTIL
+          = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testNamespaceExists() throws Exception {
+    final String NONEXISTENT_NAMESPACE = "xyzpdq_nonexistent";
+    final String EXISTING_NAMESPACE = "pdqxyz_myExistingNamespace";
+    boolean exists;
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+
+    exists = HelloHBase.namespaceExists(admin, NONEXISTENT_NAMESPACE);
+    assertEquals("#namespaceExists failed: found nonexistent namespace.",
+            false, exists);
+
+    admin.createNamespace
+        (NamespaceDescriptor.create(EXISTING_NAMESPACE).build());
+    exists = HelloHBase.namespaceExists(admin, EXISTING_NAMESPACE);
+    assertEquals("#namespaceExists failed: did NOT find existing namespace.",
+            true, exists);
+    admin.deleteNamespace(EXISTING_NAMESPACE);
+  }
+
+  @Test
+  public void testCreateNamespaceAndTable() throws Exception {
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    HelloHBase.createNamespaceAndTable(admin);
+
+    boolean namespaceExists
+            = HelloHBase.namespaceExists(admin, HelloHBase.MY_NAMESPACE_NAME);
+    assertEquals("#createNamespaceAndTable failed to create namespace.",
+            true, namespaceExists);
+
+    boolean tableExists = admin.tableExists(HelloHBase.MY_TABLE_NAME);
+    assertEquals("#createNamespaceAndTable failed to create table.",
+            true, tableExists);
+
+    admin.disableTable(HelloHBase.MY_TABLE_NAME);
+    admin.deleteTable(HelloHBase.MY_TABLE_NAME);
+    admin.deleteNamespace(HelloHBase.MY_NAMESPACE_NAME);
+  }
+
+  @Test
+  public void testPutRowToTable() throws IOException {
+    Admin admin = TEST_UTIL.getAdmin();
+    admin.createNamespace
+        (NamespaceDescriptor.create(HelloHBase.MY_NAMESPACE_NAME).build());
+    Table table
+            = TEST_UTIL.createTable
+                (HelloHBase.MY_TABLE_NAME, HelloHBase.MY_COLUMN_FAMILY_NAME);
+
+    HelloHBase.putRowToTable(table);
+    Result row = table.get(new Get(HelloHBase.MY_ROW_ID));
+    assertEquals("#putRowToTable failed to store row.", false, row.isEmpty());
+
+    TEST_UTIL.deleteTable(HelloHBase.MY_TABLE_NAME);
+    admin.deleteNamespace(HelloHBase.MY_NAMESPACE_NAME);
+  }
+
+  @Test
+  public void testDeleteRow() throws IOException {
+    Admin admin = TEST_UTIL.getAdmin();
+    admin.createNamespace
+        (NamespaceDescriptor.create(HelloHBase.MY_NAMESPACE_NAME).build());
+    Table table
+            = TEST_UTIL.createTable
+                (HelloHBase.MY_TABLE_NAME, HelloHBase.MY_COLUMN_FAMILY_NAME);
+
+    table.put(new Put(HelloHBase.MY_ROW_ID).
+            addColumn(HelloHBase.MY_COLUMN_FAMILY_NAME,
+                    HelloHBase.MY_FIRST_COLUMN_QUALIFIER,
+                    Bytes.toBytes("xyz")));
+    HelloHBase.deleteRow(table);
+    Result row = table.get(new Get(HelloHBase.MY_ROW_ID));
+    assertEquals("#deleteRow failed to delete row.", true, row.isEmpty());
+
+    TEST_UTIL.deleteTable(HelloHBase.MY_TABLE_NAME);
+    admin.deleteNamespace(HelloHBase.MY_NAMESPACE_NAME);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/83297f66/hbase-archetypes/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/pom.xml b/hbase-archetypes/pom.xml
index f011c2e..0c6c5d1 100644
--- a/hbase-archetypes/pom.xml
+++ b/hbase-archetypes/pom.xml
@@ -36,8 +36,9 @@
 
   <modules>
     <module>hbase-client-project</module>
+    <module>hbase-shaded-client-project</module>
     <!-- For new archetype, add exemplar project above this comment.
-         (See hbase-archetypes/README.txt for details on adding new archetype.)  -->
+         (See hbase-archetypes/README.md for details on adding new archetype.)  -->
     <module>hbase-archetype-builder</module>
   </modules>
 


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

Posted by en...@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-7912
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.


[18/50] [abbrv] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by en...@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


[50/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)

Posted by en...@apache.org.
HBASE-14030 HBase Backup/Restore Phase 1 (v42)


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

Branch: refs/heads/HBASE-7912
Commit: ab491d4a27984a7cdde888842978b40a5506542b
Parents: 7c54525
Author: Enis Soztutar <en...@apache.org>
Authored: Mon Mar 21 19:40:55 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Mon Mar 21 19:40:55 2016 -0700

----------------------------------------------------------------------
 bin/hbase                                       |    6 +
 .../hadoop/hbase/backup/BackupUtility.java      |  169 +
 .../org/apache/hadoop/hbase/HConstants.java     |    8 +
 .../apache/hadoop/hbase/backup/BackupType.java  |   23 +
 hbase-protocol/pom.xml                          |    1 +
 .../hbase/protobuf/generated/BackupProtos.java  | 9143 ++++++++++++++++++
 hbase-protocol/src/main/protobuf/Backup.proto   |  105 +
 hbase-server/pom.xml                            |   10 +
 .../hadoop/hbase/backup/BackupClient.java       |   40 +
 .../hadoop/hbase/backup/BackupDriver.java       |  119 +
 .../hbase/backup/BackupRestoreFactory.java      |   95 +
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  472 +
 .../hadoop/hbase/backup/RestoreClient.java      |   46 +
 .../hadoop/hbase/backup/RestoreDriver.java      |  174 +
 .../hbase/backup/impl/BackupClientImpl.java     |  183 +
 .../hbase/backup/impl/BackupCommands.java       |  158 +
 .../hadoop/hbase/backup/impl/BackupContext.java |  382 +
 .../hbase/backup/impl/BackupCopyService.java    |   37 +
 .../hbase/backup/impl/BackupException.java      |   85 +
 .../hadoop/hbase/backup/impl/BackupHandler.java |  702 ++
 .../hadoop/hbase/backup/impl/BackupManager.java |  512 +
 .../hbase/backup/impl/BackupManifest.java       |  762 ++
 .../backup/impl/BackupRestoreConstants.java     |   46 +
 .../hbase/backup/impl/BackupSnapshotCopy.java   |   42 +
 .../hadoop/hbase/backup/impl/BackupStatus.java  |  105 +
 .../hbase/backup/impl/BackupSystemTable.java    |  571 ++
 .../backup/impl/BackupSystemTableHelper.java    |  325 +
 .../hadoop/hbase/backup/impl/BackupUtil.java    |  453 +
 .../backup/impl/IncrementalBackupManager.java   |  304 +
 .../backup/impl/IncrementalRestoreService.java  |   34 +
 .../hbase/backup/impl/RestoreClientImpl.java    |  313 +
 .../hadoop/hbase/backup/impl/RestoreUtil.java   |  298 +
 .../mapreduce/MapReduceBackupCopyService.java   |  297 +
 .../mapreduce/MapReduceRestoreService.java      |   74 +
 .../hbase/backup/master/BackupLogCleaner.java   |  119 +
 .../master/LogRollMasterProcedureManager.java   |  129 +
 .../regionserver/LogRollBackupSubprocedure.java |  143 +
 .../LogRollBackupSubprocedurePool.java          |  137 +
 .../LogRollRegionServerProcedureManager.java    |  168 +
 .../BaseCoordinatedStateManager.java            |   17 +
 .../coordination/ZkCoordinatedStateManager.java |   20 +-
 .../hadoop/hbase/mapreduce/WALPlayer.java       |   54 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |    2 +
 .../procedure/RegionServerProcedureManager.java |    2 +-
 .../RegionServerProcedureManagerHost.java       |    3 +-
 .../procedure/ZKProcedureCoordinatorRpcs.java   |    2 +-
 .../hbase/procedure/ZKProcedureMemberRpcs.java  |   82 +-
 .../RegionServerFlushTableProcedureManager.java |    2 +-
 .../hbase/regionserver/HRegionServer.java       |    4 +-
 .../snapshot/RegionServerSnapshotManager.java   |    2 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |   67 +-
 .../org/apache/hadoop/hbase/util/LogUtils.java  |   43 +
 .../hadoop/hbase/wal/DefaultWALProvider.java    |    7 +-
 .../hadoop/hbase/backup/TestBackupBase.java     |  209 +
 .../hbase/backup/TestBackupBoundaryTests.java   |  105 +
 .../hbase/backup/TestBackupLogCleaner.java      |  160 +
 .../hbase/backup/TestBackupSystemTable.java     |  350 +
 .../hadoop/hbase/backup/TestFullBackup.java     |   67 +
 .../hadoop/hbase/backup/TestFullRestore.java    |  172 +
 .../hbase/backup/TestIncrementalBackup.java     |  177 +
 .../hadoop/hbase/backup/TestRemoteBackup.java   |   45 +
 .../hadoop/hbase/backup/TestRemoteRestore.java  |   55 +
 .../hbase/backup/TestRestoreBoundaryTests.java  |   86 +
 .../procedure/SimpleRSProcedureManager.java     |    2 +-
 64 files changed, 18436 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 7faaa26..83b91c0 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -101,6 +101,8 @@ if [ $# = 0 ]; then
   echo "  ltt             Run LoadTestTool"
   echo "  canary          Run the Canary tool"
   echo "  version         Print the version"
+  echo "  backup          backup tables for recovery"
+  echo "  restore         restore tables from existing backup image"
   echo "  CLASSNAME       Run the class named CLASSNAME"
   exit 1
 fi
@@ -313,6 +315,10 @@ elif [ "$COMMAND" = "hfile" ] ; then
   CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter'
 elif [ "$COMMAND" = "zkcli" ] ; then
   CLASS="org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer"
+elif [ "$COMMAND" = "backup" ] ; then
+  CLASS='org.apache.hadoop.hbase.backup.BackupDriver'
+elif [ "$COMMAND" = "restore" ] ; then
+  CLASS='org.apache.hadoop.hbase.backup.RestoreDriver'
 elif [ "$COMMAND" = "upgrade" ] ; then
   echo "This command was used to upgrade to HBase 0.96, it was removed in HBase 2.0.0."
   echo "Please follow the documentation at http://hbase.apache.org/book.html#upgrading."

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java
new file mode 100644
index 0000000..3e96f66
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java
@@ -0,0 +1,169 @@
+/**
+ * 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.backup;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.TreeMap;
+
+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.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A collection of methods used by multiple classes to backup HBase tables.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupUtility {
+  protected static final Log LOG = LogFactory.getLog(BackupUtility.class);
+  public static final String LOGNAME_SEPARATOR = ".";
+
+  private BackupUtility(){
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Check whether the backup path exist
+   * @param backupStr backup
+   * @param conf configuration
+   * @return Yes if path exists
+   * @throws IOException exception
+   */
+  public static boolean checkPathExist(String backupStr, Configuration conf)
+    throws IOException {
+    boolean isExist = false;
+    Path backupPath = new Path(backupStr);
+    FileSystem fileSys = backupPath.getFileSystem(conf);
+    String targetFsScheme = fileSys.getUri().getScheme();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme);
+    }
+    if (fileSys.exists(backupPath)) {
+      isExist = true;
+    }
+    return isExist;
+  }
+
+  // check target path first, confirm it doesn't exist before backup
+  public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException {
+    boolean targetExists = false;
+    try {
+      targetExists = checkPathExist(backupRootPath, conf);
+    } catch (IOException e) {
+      String expMsg = e.getMessage();
+      String newMsg = null;
+      if (expMsg.contains("No FileSystem for scheme")) {
+        newMsg =
+            "Unsupported filesystem scheme found in the backup target url. Error Message: "
+                + newMsg;
+        LOG.error(newMsg);
+        throw new IOException(newMsg);
+      } else {
+        throw e;
+      }
+    } catch (RuntimeException e) {
+      LOG.error(e.getMessage());
+      throw e;
+    }
+
+    if (targetExists) {
+      LOG.info("Using existing backup root dir: " + backupRootPath);
+    } else {
+      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
+    }
+  }
+
+  /**
+   * Get the min value for all the Values a map.
+   * @param map map
+   * @return the min value
+   */
+  public static <T> Long getMinValue(HashMap<T, Long> map) {
+    Long minTimestamp = null;
+    if (map != null) {
+      ArrayList<Long> timestampList = new ArrayList<Long>(map.values());
+      Collections.sort(timestampList);
+      // The min among all the RS log timestamps will be kept in hbase:backup table.
+      minTimestamp = timestampList.get(0);
+    }
+    return minTimestamp;
+  }
+
+  /**
+   * TODO: verify the code
+   * @param p path
+   * @return host name
+   * @throws IOException exception
+   */
+  public static String parseHostFromOldLog(Path p) throws IOException {
+    String n = p.getName();
+    int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
+    String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
+    return ServerName.parseHostname(s) + ":" + ServerName.parsePort(s);
+  }
+
+  /**
+   * Given the log file, parse the timestamp from the file name. The timestamp is the last number.
+   * @param p a path to the log file
+   * @return the timestamp
+   * @throws IOException exception
+   */
+  public static Long getCreationTime(Path p) throws IOException {
+    int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR);
+    if (idx < 0) {
+      throw new IOException("Cannot parse timestamp from path " + p);
+    }
+    String ts = p.getName().substring(idx + 1);
+    return Long.parseLong(ts);
+  }
+
+  public static List<String> getFiles(FileSystem fs, Path rootDir, List<String> files,
+    PathFilter filter) throws FileNotFoundException, IOException {
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(rootDir, true);
+
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.isDirectory()) {
+        continue;
+      }
+      // apply filter
+      if (filter.accept(lfs.getPath())) {
+        files.add(lfs.getPath().toString());
+      }
+    }
+    return files;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index a02d89a..59948ca 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1260,6 +1260,14 @@ public final class HConstants {
   public static final String DEFAULT_TEMPORARY_HDFS_DIRECTORY = "/user/"
       + System.getProperty("user.name") + "/hbase-staging";
 
+  /**
+   * Backup/Restore constants
+   */
+  public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable";
+  public final static boolean BACKUP_ENABLE_DEFAULT = true;
+  public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl";
+  public final static int BACKUP_SYSTEM_TTL_DEFAULT = FOREVER;
+
   private HConstants() {
     // Can't be instantiated with this ctor.
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java
new file mode 100644
index 0000000..e2e3446
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java
@@ -0,0 +1,23 @@
+/**
+ * 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.backup;
+
+public enum BackupType {
+  FULL, INCREMENTAL
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index 8034576..9098944 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -171,6 +171,7 @@
                           <include>Admin.proto</include>
                           <include>Aggregate.proto</include>
                           <include>Authentication.proto</include>
+                          <include>Backup.proto</include>
                           <include>Cell.proto</include>
                           <include>Client.proto</include>
                           <include>ClusterId.proto</include>


[04/50] [abbrv] hbase git commit: HBASE-15016 Services a Store needs from a Region

Posted by en...@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-7912
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();


[10/50] [abbrv] hbase git commit: HBASE-15222 Use less contended classes for metrics

Posted by en...@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;


[31/50] [abbrv] hbase git commit: HBASE-15349 Update surefire version to 2.19.1. (Apekshit)

Posted by en...@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-7912
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>


[09/50] [abbrv] hbase git commit: HBASE-15310 hbase-spark module has compilation failures with clover profile

Posted by en...@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-7912
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>
 


[44/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
new file mode 100644
index 0000000..21bf63c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
@@ -0,0 +1,105 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupBoundaryTests extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupBoundaryTests.class);
+
+  /**
+   * Verify that full backup is created on a single empty table correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupSingleEmpty() throws Exception {
+
+    LOG.info("create full backup image on single table");
+    List<TableName> tables = Lists.newArrayList(table3);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    LOG.info("Finished Backup");
+    assertTrue(checkSucceeded(backupId));
+  }
+
+  /**
+   * Verify that full backup is created on multiple empty tables correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupMultipleEmpty() throws Exception {
+    LOG.info("create full backup image on mulitple empty tables");
+
+    List<TableName> tables = Lists.newArrayList(table3, table4);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+  }
+
+  /**
+   * Verify that full backup fails on a single table that does not exist.
+   * @throws Exception
+   */
+  @Test(expected = DoNotRetryIOException.class)
+  public void testFullBackupSingleDNE() throws Exception {
+
+    LOG.info("test full backup fails on a single table that does not exist");
+    List<TableName> tables = toList("tabledne");
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+  }
+
+  /**
+   * Verify that full backup fails on multiple tables that do not exist.
+   * @throws Exception
+   */
+  @Test(expected = DoNotRetryIOException.class)
+  public void testFullBackupMultipleDNE() throws Exception {
+
+    LOG.info("test full backup fails on multiple tables that do not exist");
+    List<TableName> tables = toList("table1dne", "table2dne");
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+  }
+
+  /**
+   * Verify that full backup fails on tableset containing real and fake tables.
+   * @throws Exception
+   */
+  @Test(expected = DoNotRetryIOException.class)
+  public void testFullBackupMixExistAndDNE() throws Exception {
+    LOG.info("create full backup fails on tableset containing real and fake table");
+
+    List<TableName> tables = toList(table1.getNameAsString(), "tabledne");
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    //assertTrue(checkSucceeded(backupId)); // TODO
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
new file mode 100644
index 0000000..899f53b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
@@ -0,0 +1,160 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+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.wal.DefaultWALProvider;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupLogCleaner extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestBackupLogCleaner.class);
+
+  // implements all test cases in 1 test since incremental full backup/
+  // incremental backup has dependencies
+  @Test
+  public void testBackupLogCleaner() throws Exception {
+
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+
+    List<TableName> tableSetFullList = Lists.newArrayList(table1, table2, table3, table4);
+
+    try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+        BackupSystemTable systemTable = new BackupSystemTable(connection)) {
+      // Verify that we have no backup sessions yet
+      assertFalse(systemTable.hasBackupSessions());
+
+      List<FileStatus> walFiles = getListOfWALFiles(TEST_UTIL.getConfiguration());
+      List<String> swalFiles = convert(walFiles);
+      BackupLogCleaner cleaner = new BackupLogCleaner();
+      cleaner.setConf(TEST_UTIL.getConfiguration());
+
+      Iterable<FileStatus> deletable = cleaner.getDeletableFiles(walFiles);
+      // We can delete all files because we do not have yet recorded backup sessions
+      assertTrue(Iterables.size(deletable) == walFiles.size());
+
+      systemTable.addWALFiles(swalFiles, "backup");
+      String backupIdFull = getBackupClient().create(BackupType.FULL, tableSetFullList,
+        BACKUP_ROOT_DIR);
+      assertTrue(checkSucceeded(backupIdFull));
+      // Check one more time
+      deletable = cleaner.getDeletableFiles(walFiles);
+      // We can delete wal files because they were saved into hbase:backup table
+      int size = Iterables.size(deletable);
+      assertTrue(size == walFiles.size());
+
+      List<FileStatus> newWalFiles = getListOfWALFiles(TEST_UTIL.getConfiguration());
+      LOG.debug("WAL list after full backup");
+      convert(newWalFiles);
+
+      // New list of wal files is greater than the previous one,
+      // because new wal per RS have been opened after full backup
+      assertTrue(walFiles.size() < newWalFiles.size());
+      // TODO : verify that result files are not walFiles collection
+      Connection conn = ConnectionFactory.createConnection(conf1);
+      // #2 - insert some data to table
+      HTable t1 = (HTable) conn.getTable(table1);
+      Put p1;
+      for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+        p1 = new Put(Bytes.toBytes("row-t1" + i));
+        p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+        t1.put(p1);
+      }
+
+      t1.close();
+
+      HTable t2 = (HTable) conn.getTable(table2);
+      Put p2;
+      for (int i = 0; i < 5; i++) {
+        p2 = new Put(Bytes.toBytes("row-t2" + i));
+        p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+        t2.put(p2);
+      }
+
+      t2.close();
+
+      // #3 - incremental backup for multiple tables
+
+      List<TableName> tableSetIncList = Lists.newArrayList(table1, table2, table3);
+      String backupIdIncMultiple =
+          getBackupClient().create(BackupType.INCREMENTAL, tableSetIncList, BACKUP_ROOT_DIR);
+      assertTrue(checkSucceeded(backupIdIncMultiple));
+      deletable = cleaner.getDeletableFiles(newWalFiles);
+
+      assertTrue(Iterables.size(deletable) == newWalFiles.size());
+
+      conn.close();
+    }
+  }
+
+  private List<String> convert(List<FileStatus> walFiles) {
+    List<String> result = new ArrayList<String>();
+    for (FileStatus fs : walFiles) {
+      LOG.debug("+++WAL: " + fs.getPath().toString());
+      result.add(fs.getPath().toString());
+    }
+    return result;
+  }
+
+  private List<FileStatus> getListOfWALFiles(Configuration c) throws IOException {
+    Path logRoot = new Path(FSUtils.getRootDir(c), HConstants.HREGION_LOGDIR_NAME);
+    FileSystem fs = FileSystem.get(c);
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(logRoot, true);
+    List<FileStatus> logFiles = new ArrayList<FileStatus>();
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.isFile() && !DefaultWALProvider.isMetaFile(lfs.getPath())) {
+        logFiles.add(lfs);
+        LOG.info(lfs);
+      }
+    }
+    return logFiles;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
new file mode 100644
index 0000000..2dc31df
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
@@ -0,0 +1,350 @@
+/**
+ *
+ * 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.backup;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupContext;
+import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test cases for hbase:backup API
+ *
+ */
+@Category(MediumTests.class)
+public class TestBackupSystemTable {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  protected static Configuration conf = UTIL.getConfiguration();
+  protected static MiniHBaseCluster cluster;
+  protected static Connection conn;
+  protected BackupSystemTable table;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = UTIL.startMiniCluster();
+    conn = ConnectionFactory.createConnection(UTIL.getConfiguration());
+  }
+
+  @Before
+  public void before() throws IOException {
+    table = new BackupSystemTable(conn);
+  }
+
+  @After
+  public void after() {
+    if (table != null) {
+      table.close();
+    }
+  }
+
+  @Test
+  public void testUpdateReadDeleteBackupStatus() throws IOException {
+    BackupContext ctx = createBackupContext();
+    table.updateBackupStatus(ctx);
+    BackupContext readCtx = table.readBackupStatus(ctx.getBackupId());
+    assertTrue(compare(ctx, readCtx));
+
+    // try fake backup id
+    readCtx = table.readBackupStatus("fake");
+
+    assertNull(readCtx);
+    // delete backup context
+    table.deleteBackupStatus(ctx.getBackupId());
+    readCtx = table.readBackupStatus(ctx.getBackupId());
+    assertNull(readCtx);
+    cleanBackupTable();
+  }
+
+  @Test
+  public void testWriteReadBackupStartCode() throws IOException {
+    Long code = 100L;
+    table.writeBackupStartCode(code);
+    String readCode = table.readBackupStartCode();
+    assertEquals(code, new Long(Long.parseLong(readCode)));
+    cleanBackupTable();
+  }
+
+  private void cleanBackupTable() throws IOException {
+    Admin admin = UTIL.getHBaseAdmin();
+    admin.disableTable(BackupSystemTable.getTableName());
+    admin.truncateTable(BackupSystemTable.getTableName(), true);
+    if (admin.isTableDisabled(BackupSystemTable.getTableName())) {
+      admin.enableTable(BackupSystemTable.getTableName());
+    }
+  }
+
+  @Test
+  public void testBackupHistory() throws IOException {
+    int n = 10;
+    List<BackupContext> list = createBackupContextList(n);
+
+    // Load data
+    for (BackupContext bc : list) {
+      // Make sure we set right status
+      bc.setState(BackupState.COMPLETE);
+      table.updateBackupStatus(bc);
+    }
+
+    // Reverse list for comparison
+    Collections.reverse(list);
+    ArrayList<BackupCompleteData> history = table.getBackupHistory();
+    assertTrue(history.size() == n);
+
+    for (int i = 0; i < n; i++) {
+      BackupContext ctx = list.get(i);
+      BackupCompleteData data = history.get(i);
+      assertTrue(compare(ctx, data));
+    }
+
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testRegionServerLastLogRollResults() throws IOException {
+    String[] servers = new String[] { "server1", "server2", "server3" };
+    Long[] timestamps = new Long[] { 100L, 102L, 107L };
+
+    for (int i = 0; i < servers.length; i++) {
+      table.writeRegionServerLastLogRollResult(servers[i], timestamps[i]);
+    }
+
+    HashMap<String, Long> result = table.readRegionServerLastLogRollResult();
+    assertTrue(servers.length == result.size());
+    Set<String> keys = result.keySet();
+    String[] keysAsArray = new String[keys.size()];
+    keys.toArray(keysAsArray);
+    Arrays.sort(keysAsArray);
+
+    for (int i = 0; i < keysAsArray.length; i++) {
+      assertEquals(keysAsArray[i], servers[i]);
+      Long ts1 = timestamps[i];
+      Long ts2 = result.get(keysAsArray[i]);
+      assertEquals(ts1, ts2);
+    }
+
+    cleanBackupTable();
+  }
+
+  @Test
+  public void testIncrementalBackupTableSet() throws IOException {
+    TreeSet<TableName> tables1 = new TreeSet<>();
+
+    tables1.add(TableName.valueOf("t1"));
+    tables1.add(TableName.valueOf("t2"));
+    tables1.add(TableName.valueOf("t3"));
+
+    TreeSet<TableName> tables2 = new TreeSet<>();
+
+    tables2.add(TableName.valueOf("t3"));
+    tables2.add(TableName.valueOf("t4"));
+    tables2.add(TableName.valueOf("t5"));
+
+    table.addIncrementalBackupTableSet(tables1);
+    TreeSet<TableName> res1 = (TreeSet<TableName>) table.getIncrementalBackupTableSet();
+    assertTrue(tables1.size() == res1.size());
+    Iterator<TableName> desc1 = tables1.descendingIterator();
+    Iterator<TableName> desc2 = res1.descendingIterator();
+    while (desc1.hasNext()) {
+      assertEquals(desc1.next(), desc2.next());
+    }
+
+    table.addIncrementalBackupTableSet(tables2);
+    TreeSet<TableName> res2 = (TreeSet<TableName>) table.getIncrementalBackupTableSet();
+    assertTrue((tables2.size() + tables1.size() - 1) == res2.size());
+
+    tables1.addAll(tables2);
+
+    desc1 = tables1.descendingIterator();
+    desc2 = res2.descendingIterator();
+
+    while (desc1.hasNext()) {
+      assertEquals(desc1.next(), desc2.next());
+    }
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testRegionServerLogTimestampMap() throws IOException {
+    TreeSet<TableName> tables = new TreeSet<>();
+
+    tables.add(TableName.valueOf("t1"));
+    tables.add(TableName.valueOf("t2"));
+    tables.add(TableName.valueOf("t3"));
+
+    HashMap<String, Long> rsTimestampMap = new HashMap<String, Long>();
+
+    rsTimestampMap.put("rs1", 100L);
+    rsTimestampMap.put("rs2", 101L);
+    rsTimestampMap.put("rs3", 103L);
+
+    table.writeRegionServerLogTimestamp(tables, rsTimestampMap);
+
+    HashMap<TableName, HashMap<String, Long>> result = table.readLogTimestampMap();
+
+    assertTrue(tables.size() == result.size());
+
+    for (TableName t : tables) {
+      HashMap<String, Long> rstm = result.get(t);
+      assertNotNull(rstm);
+      assertEquals(rstm.get("rs1"), new Long(100L));
+      assertEquals(rstm.get("rs2"), new Long(101L));
+      assertEquals(rstm.get("rs3"), new Long(103L));
+    }
+
+    Set<TableName> tables1 = new TreeSet<>();
+
+    tables1.add(TableName.valueOf("t3"));
+    tables1.add(TableName.valueOf("t4"));
+    tables1.add(TableName.valueOf("t5"));
+
+    HashMap<String, Long> rsTimestampMap1 = new HashMap<String, Long>();
+
+    rsTimestampMap1.put("rs1", 200L);
+    rsTimestampMap1.put("rs2", 201L);
+    rsTimestampMap1.put("rs3", 203L);
+
+    table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1);
+
+    result = table.readLogTimestampMap();
+
+    assertTrue(5 == result.size());
+
+    for (TableName t : tables) {
+      HashMap<String, Long> rstm = result.get(t);
+      assertNotNull(rstm);
+      if (t.equals(TableName.valueOf("t3")) == false) {
+        assertEquals(rstm.get("rs1"), new Long(100L));
+        assertEquals(rstm.get("rs2"), new Long(101L));
+        assertEquals(rstm.get("rs3"), new Long(103L));
+      } else {
+        assertEquals(rstm.get("rs1"), new Long(200L));
+        assertEquals(rstm.get("rs2"), new Long(201L));
+        assertEquals(rstm.get("rs3"), new Long(203L));
+      }
+    }
+
+    for (TableName t : tables1) {
+      HashMap<String, Long> rstm = result.get(t);
+      assertNotNull(rstm);
+      assertEquals(rstm.get("rs1"), new Long(200L));
+      assertEquals(rstm.get("rs2"), new Long(201L));
+      assertEquals(rstm.get("rs3"), new Long(203L));
+    }
+
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testAddWALFiles() throws IOException {
+    List<String> files =
+        Arrays.asList("hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.1",
+          "hdfs://server/WALs/srv2,102,16666/srv2,102,16666.default.2",
+            "hdfs://server/WALs/srv3,103,17777/srv3,103,17777.default.3");
+    String newFile = "hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.5";
+
+    table.addWALFiles(files, "backup");
+
+    assertTrue(table.checkWALFile(files.get(0)));
+    assertTrue(table.checkWALFile(files.get(1)));
+    assertTrue(table.checkWALFile(files.get(2)));
+    assertFalse(table.checkWALFile(newFile));
+
+    cleanBackupTable();
+  }
+
+  private boolean compare(BackupContext ctx, BackupCompleteData data) {
+
+    return ctx.getBackupId().equals(data.getBackupToken())
+        && ctx.getTargetRootDir().equals(data.getBackupRootPath())
+        && ctx.getType().toString().equals(data.getType())
+        && ctx.getStartTs() == Long.parseLong(data.getStartTime())
+        && ctx.getEndTs() == Long.parseLong(data.getEndTime());
+
+  }
+
+  private boolean compare(BackupContext one, BackupContext two) {
+    return one.getBackupId().equals(two.getBackupId()) && one.getType().equals(two.getType())
+        && one.getTargetRootDir().equals(two.getTargetRootDir())
+        && one.getStartTs() == two.getStartTs() && one.getEndTs() == two.getEndTs();
+  }
+
+  private BackupContext createBackupContext() {
+
+    BackupContext ctxt =
+        new BackupContext("backup_" + System.nanoTime(), BackupType.FULL,
+          new TableName[] {
+              TableName.valueOf("t1"), TableName.valueOf("t2"), TableName.valueOf("t3") },
+          "/hbase/backup");
+    ctxt.setStartTs(System.currentTimeMillis());
+    ctxt.setEndTs(System.currentTimeMillis() + 1);
+    return ctxt;
+  }
+
+  private List<BackupContext> createBackupContextList(int size) {
+    List<BackupContext> list = new ArrayList<BackupContext>();
+    for (int i = 0; i < size; i++) {
+      list.add(createBackupContext());
+      try {
+        Thread.sleep(10);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+    }
+    return list;
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    if (cluster != null) cluster.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
new file mode 100644
index 0000000..d9bade1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
@@ -0,0 +1,67 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestFullBackup extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullBackup.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupSingle() throws Exception {
+    LOG.info("test full backup on a single table with data");
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+  }
+
+  /**
+   * Verify that full backup is created on multiple tables correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupMultiple() throws Exception {
+    LOG.info("create full backup image on multiple tables with data");
+    List<TableName> tables = Lists.newArrayList(table1, table1);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+  }
+
+  /**
+   * Verify that full backup is created on all tables correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupAll() throws Exception {
+    LOG.info("create full backup image on all tables");
+    String backupId = getBackupClient().create(BackupType.FULL, null, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
new file mode 100644
index 0000000..e4c4a07
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
@@ -0,0 +1,172 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestFullRestore extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullRestore.class);
+
+  /**
+   * Verify that a single table is restored to a new table
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingle() throws Exception {
+
+    LOG.info("test full restore on a single table empty table");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+
+    TableName[] tableset = new TableName[] { table1 };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap, false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1_restore));
+    TEST_UTIL.deleteTable(table1_restore);
+    hba.close();
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultiple() throws Exception {
+    LOG.info("create full backup image on multiple tables");
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset = new TableName[] { table2, table3 };
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupId, false, false,
+      restore_tableset, tablemap, false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table2_restore));
+    assertTrue(hba.tableExists(table3_restore));
+    TEST_UTIL.deleteTable(table2_restore);
+    TEST_UTIL.deleteTable(table3_restore);
+    hba.close();
+  }
+
+  /**
+   * Verify that a single table is restored using overwrite
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingleOverwrite() throws Exception {
+
+    LOG.info("test full restore on a single table empty table");
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+
+    TableName[] tableset = new TableName[] { table1 };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupId, false, false, tableset, null,
+      true);
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables using overwrite.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultipleOverwrite() throws Exception {
+    LOG.info("create full backup image on multiple tables");
+
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset = new TableName[] { table2, table3 };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupId, false,
+      false, restore_tableset, null, true);
+  }
+
+  /**
+   * Verify that restore fails on a single table that does not exist.
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testFullRestoreSingleDNE() throws Exception {
+
+    LOG.info("test restore fails on a single table that does not exist");
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+
+    TableName[] tableset = new TableName[] { TableName.valueOf("faketable") };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
+      false);
+  }
+
+  /**
+   * Verify that restore fails on multiple tables that do not exist.
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testFullRestoreMultipleDNE() throws Exception {
+
+    LOG.info("test restore fails on multiple tables that do not exist");
+
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset
+      = new TableName[] { TableName.valueOf("faketable1"), TableName.valueOf("faketable2") };
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupId, false,
+      false, restore_tableset, tablemap, false);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
new file mode 100644
index 0000000..23b1af1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -0,0 +1,177 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestIncrementalBackup extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class);
+  //implement all testcases in 1 test since incremental backup/restore has dependencies
+  @Test
+  public void TestIncBackupRestore() throws Exception {
+    HBackupFileSystem hbfs;
+
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+
+    List<TableName> tables = Lists.newArrayList(table1, table2, table3, table4);
+    String backupIdFull = getBackupClient().create(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+
+    assertTrue(checkSucceeded(backupIdFull));
+
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    // #2 - insert some data to table
+    HTable t1 = (HTable) conn.getTable(table1);
+    Put p1;
+    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+      p1 = new Put(Bytes.toBytes("row-t1" + i));
+      p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t1.put(p1);
+    }
+
+    Assert.assertThat(TEST_UTIL.countRows(t1), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
+    t1.close();
+
+    HTable t2 =  (HTable) conn.getTable(table2);
+    Put p2;
+    for (int i = 0; i < 5; i++) {
+      p2 = new Put(Bytes.toBytes("row-t2" + i));
+      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t2.put(p2);
+    }
+
+    Assert.assertThat(TEST_UTIL.countRows(t2), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
+    t2.close();
+
+    // #3 - incremental backup for multiple tables
+
+
+    tables = Lists.newArrayList(table1, table2, table3);
+    String backupIdIncMultiple = getBackupClient().create(BackupType.INCREMENTAL,
+      tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupIdIncMultiple));
+
+
+    // #4 - restore full backup for all tables, without overwrite
+    TableName[] tablesRestoreFull =
+        new TableName[] { table1, table2, table3, table4 };
+
+    TableName[] tablesMapFull =
+        new TableName[] { table1_restore, table2_restore, table3_restore, table4_restore };
+
+    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdFull);
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupIdFull, false, false,
+      tablesRestoreFull,
+      tablesMapFull, false);
+
+    // #5.1 - check tables for full restore
+    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hAdmin.tableExists(table1_restore));
+    assertTrue(hAdmin.tableExists(table2_restore));
+    assertTrue(hAdmin.tableExists(table3_restore));
+    assertTrue(hAdmin.tableExists(table4_restore));
+
+    hAdmin.close();
+
+    // #5.2 - checking row count of tables for full restore
+    HTable hTable = (HTable) conn.getTable(table1_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table2_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table3_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table4_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+
+    // #6 - restore incremental backup for multiple tables, with overwrite
+    TableName[] tablesRestoreIncMultiple =
+        new TableName[] { table1, table2, table3 };
+    TableName[] tablesMapIncMultiple =
+        new TableName[] { table1_restore, table2_restore, table3_restore };
+    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncMultiple);
+    client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupIdIncMultiple, false, false,
+      tablesRestoreIncMultiple, tablesMapIncMultiple, true);
+
+    hTable = (HTable) conn.getTable(table1_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table2_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table3_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+
+    // #7 - incremental backup for single, empty table
+
+    tables = toList(table4.getNameAsString());
+    String backupIdIncEmpty =
+        getBackupClient().create(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupIdIncEmpty));
+
+
+    // #8 - restore incremental backup for single empty table, with overwrite
+    TableName[] tablesRestoreIncEmpty = new TableName[] { table4 };
+    TableName[] tablesMapIncEmpty = new TableName[] { table4_restore };
+    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncEmpty);
+
+    getRestoreClient().restore(BACKUP_ROOT_DIR, backupIdIncEmpty, false, false,
+      tablesRestoreIncEmpty,
+      tablesMapIncEmpty, true);
+
+    hTable = (HTable) conn.getTable(table4_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+    conn.close();
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
new file mode 100644
index 0000000..035188c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
@@ -0,0 +1,45 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestRemoteBackup extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class);
+
+  /**
+   * Verify that a remote full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupRemote() throws Exception {
+
+    LOG.info("test remote full backup on a single table");
+
+    String backupId =
+        getBackupClient().create(BackupType.FULL,
+          Lists.newArrayList(table1), BACKUP_REMOTE_ROOT_DIR);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
new file mode 100644
index 0000000..6a66a0d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
@@ -0,0 +1,55 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestRemoteRestore extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRemoteRestore.class);
+
+  /**
+   * Verify that a remote restore on a single table is successful.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreRemote() throws Exception {
+
+    LOG.info("test remote full backup on a single table");
+    String backupId =
+        getBackupClient().create(BackupType.FULL, toList(table1.getNameAsString()),
+          BACKUP_REMOTE_ROOT_DIR);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+    TableName[] tableset = new TableName[] { table1 };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    Path path = new Path(BACKUP_REMOTE_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    getRestoreClient().restore(BACKUP_REMOTE_ROOT_DIR, backupId, false, false, tableset,
+      tablemap, false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1_restore));
+    TEST_UTIL.deleteTable(table1_restore);
+    hba.close();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
new file mode 100644
index 0000000..6f9b3c7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
@@ -0,0 +1,86 @@
+/**
+ * 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.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestRestoreBoundaryTests extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRestoreBoundaryTests.class);
+
+  /**
+   * Verify that a single empty table is restored to a new table
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingleEmpty() throws Exception {
+    LOG.info("test full restore on a single table empty table");
+    String backupId =
+        getBackupClient().create(BackupType.FULL, toList(table1.getNameAsString()),
+            BACKUP_ROOT_DIR);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+    TableName[] tableset = new TableName[] { table1 };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    getRestoreClient().restore(BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
+      false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1_restore));
+    TEST_UTIL.deleteTable(table1_restore);
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultipleEmpty() throws Exception {
+    LOG.info("create full backup image on multiple tables");
+
+    List<TableName> tables = toList(table2.getNameAsString(), table3.getNameAsString());
+    String backupId = getBackupClient().create(BackupType.FULL, tables,BACKUP_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+    TableName[] restore_tableset = new TableName[] { table2, table3};
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    getRestoreClient().restore(BACKUP_ROOT_DIR, backupId, false, false, restore_tableset,
+      tablemap,
+      false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table2_restore));
+    assertTrue(hba.tableExists(table3_restore));
+    TEST_UTIL.deleteTable(table2_restore);
+    TEST_UTIL.deleteTable(table3_restore);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
index 7620bbb..cd2efad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
@@ -49,7 +49,7 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager {
   private ProcedureMember member;
 
   @Override
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw, getProcedureSignature());


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

Posted by en...@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-7912
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


[11/50] [abbrv] hbase git commit: HBASE-15222 Use less contended classes for metrics

Posted by en...@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-7912
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/50] [abbrv] hbase git commit: HBASE-15215 TestBlockEvictionFromClient is flaky in jdk1.7 build (setting offheap)

Posted by en...@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-7912
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


[14/50] [abbrv] hbase git commit: HBASE-15136 Explore different queuing behaviors while busy

Posted by en...@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-7912
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;
+  }
 }


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

Posted by en...@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-7912
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;
-  }
 }


[49/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
new file mode 100644
index 0000000..1a7a1ba
--- /dev/null
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
@@ -0,0 +1,9143 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: Backup.proto
+
+package org.apache.hadoop.hbase.protobuf.generated;
+
+public final class BackupProtos {
+  private BackupProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  /**
+   * Protobuf enum {@code hbase.pb.BackupType}
+   */
+  public enum BackupType
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>FULL = 0;</code>
+     */
+    FULL(0, 0),
+    /**
+     * <code>INCREMENTAL = 1;</code>
+     */
+    INCREMENTAL(1, 1),
+    ;
+
+    /**
+     * <code>FULL = 0;</code>
+     */
+    public static final int FULL_VALUE = 0;
+    /**
+     * <code>INCREMENTAL = 1;</code>
+     */
+    public static final int INCREMENTAL_VALUE = 1;
+
+
+    public final int getNumber() { return value; }
+
+    public static BackupType valueOf(int value) {
+      switch (value) {
+        case 0: return FULL;
+        case 1: return INCREMENTAL;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<BackupType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<BackupType>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<BackupType>() {
+            public BackupType findValueByNumber(int number) {
+              return BackupType.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.BackupProtos.getDescriptor().getEnumTypes().get(0);
+    }
+
+    private static final BackupType[] VALUES = values();
+
+    public static BackupType 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 BackupType(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.BackupType)
+  }
+
+  public interface BackupImageOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string backup_id = 1;
+    /**
+     * <code>required string backup_id = 1;</code>
+     */
+    boolean hasBackupId();
+    /**
+     * <code>required string backup_id = 1;</code>
+     */
+    java.lang.String getBackupId();
+    /**
+     * <code>required string backup_id = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getBackupIdBytes();
+
+    // required .hbase.pb.BackupType backup_type = 2;
+    /**
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    boolean hasBackupType();
+    /**
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType();
+
+    // required string root_dir = 3;
+    /**
+     * <code>required string root_dir = 3;</code>
+     */
+    boolean hasRootDir();
+    /**
+     * <code>required string root_dir = 3;</code>
+     */
+    java.lang.String getRootDir();
+    /**
+     * <code>required string root_dir = 3;</code>
+     */
+    com.google.protobuf.ByteString
+        getRootDirBytes();
+
+    // repeated .hbase.pb.TableName table_list = 4;
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> 
+        getTableListList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index);
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    int getTableListCount();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTableListOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
+        int index);
+
+    // required uint64 start_ts = 5;
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    boolean hasStartTs();
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    long getStartTs();
+
+    // required uint64 complete_ts = 6;
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    boolean hasCompleteTs();
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    long getCompleteTs();
+
+    // repeated .hbase.pb.BackupImage ancestors = 7;
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> 
+        getAncestorsList();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index);
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    int getAncestorsCount();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
+        getAncestorsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.BackupImage}
+   */
+  public static final class BackupImage extends
+      com.google.protobuf.GeneratedMessage
+      implements BackupImageOrBuilder {
+    // Use BackupImage.newBuilder() to construct.
+    private BackupImage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private BackupImage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final BackupImage defaultInstance;
+    public static BackupImage getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public BackupImage getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private BackupImage(
+        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 10: {
+              bitField0_ |= 0x00000001;
+              backupId_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(2, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                backupType_ = value;
+              }
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              rootDir_ = input.readBytes();
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+                tableList_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>();
+                mutable_bitField0_ |= 0x00000008;
+              }
+              tableList_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              startTs_ = input.readUInt64();
+              break;
+            }
+            case 48: {
+              bitField0_ |= 0x00000010;
+              completeTs_ = input.readUInt64();
+              break;
+            }
+            case 58: {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                ancestors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage>();
+                mutable_bitField0_ |= 0x00000040;
+              }
+              ancestors_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.PARSER, extensionRegistry));
+              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 {
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          tableList_ = java.util.Collections.unmodifiableList(tableList_);
+        }
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+          ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<BackupImage> PARSER =
+        new com.google.protobuf.AbstractParser<BackupImage>() {
+      public BackupImage parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new BackupImage(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<BackupImage> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string backup_id = 1;
+    public static final int BACKUP_ID_FIELD_NUMBER = 1;
+    private java.lang.Object backupId_;
+    /**
+     * <code>required string backup_id = 1;</code>
+     */
+    public boolean hasBackupId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string backup_id = 1;</code>
+     */
+    public java.lang.String getBackupId() {
+      java.lang.Object ref = backupId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          backupId_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string backup_id = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getBackupIdBytes() {
+      java.lang.Object ref = backupId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        backupId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required .hbase.pb.BackupType backup_type = 2;
+    public static final int BACKUP_TYPE_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType backupType_;
+    /**
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    public boolean hasBackupType() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType() {
+      return backupType_;
+    }
+
+    // required string root_dir = 3;
+    public static final int ROOT_DIR_FIELD_NUMBER = 3;
+    private java.lang.Object rootDir_;
+    /**
+     * <code>required string root_dir = 3;</code>
+     */
+    public boolean hasRootDir() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required string root_dir = 3;</code>
+     */
+    public java.lang.String getRootDir() {
+      java.lang.Object ref = rootDir_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          rootDir_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string root_dir = 3;</code>
+     */
+    public com.google.protobuf.ByteString
+        getRootDirBytes() {
+      java.lang.Object ref = rootDir_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        rootDir_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated .hbase.pb.TableName table_list = 4;
+    public static final int TABLE_LIST_FIELD_NUMBER = 4;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableList_;
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableListList() {
+      return tableList_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTableListOrBuilderList() {
+      return tableList_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public int getTableListCount() {
+      return tableList_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index) {
+      return tableList_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
+        int index) {
+      return tableList_.get(index);
+    }
+
+    // required uint64 start_ts = 5;
+    public static final int START_TS_FIELD_NUMBER = 5;
+    private long startTs_;
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    public boolean hasStartTs() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    public long getStartTs() {
+      return startTs_;
+    }
+
+    // required uint64 complete_ts = 6;
+    public static final int COMPLETE_TS_FIELD_NUMBER = 6;
+    private long completeTs_;
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    public boolean hasCompleteTs() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    public long getCompleteTs() {
+      return completeTs_;
+    }
+
+    // repeated .hbase.pb.BackupImage ancestors = 7;
+    public static final int ANCESTORS_FIELD_NUMBER = 7;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> ancestors_;
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> getAncestorsList() {
+      return ancestors_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
+        getAncestorsOrBuilderList() {
+      return ancestors_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public int getAncestorsCount() {
+      return ancestors_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) {
+      return ancestors_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
+        int index) {
+      return ancestors_.get(index);
+    }
+
+    private void initFields() {
+      backupId_ = "";
+      backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
+      rootDir_ = "";
+      tableList_ = java.util.Collections.emptyList();
+      startTs_ = 0L;
+      completeTs_ = 0L;
+      ancestors_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasBackupId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasBackupType()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRootDir()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStartTs()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasCompleteTs()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getTableListCount(); i++) {
+        if (!getTableList(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getAncestorsCount(); i++) {
+        if (!getAncestors(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getBackupIdBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeEnum(2, backupType_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getRootDirBytes());
+      }
+      for (int i = 0; i < tableList_.size(); i++) {
+        output.writeMessage(4, tableList_.get(i));
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(5, startTs_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(6, completeTs_);
+      }
+      for (int i = 0; i < ancestors_.size(); i++) {
+        output.writeMessage(7, ancestors_.get(i));
+      }
+      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
+          .computeBytesSize(1, getBackupIdBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(2, backupType_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getRootDirBytes());
+      }
+      for (int i = 0; i < tableList_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, tableList_.get(i));
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(5, startTs_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(6, completeTs_);
+      }
+      for (int i = 0; i < ancestors_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(7, ancestors_.get(i));
+      }
+      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.BackupProtos.BackupImage)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage other = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) obj;
+
+      boolean result = true;
+      result = result && (hasBackupId() == other.hasBackupId());
+      if (hasBackupId()) {
+        result = result && getBackupId()
+            .equals(other.getBackupId());
+      }
+      result = result && (hasBackupType() == other.hasBackupType());
+      if (hasBackupType()) {
+        result = result &&
+            (getBackupType() == other.getBackupType());
+      }
+      result = result && (hasRootDir() == other.hasRootDir());
+      if (hasRootDir()) {
+        result = result && getRootDir()
+            .equals(other.getRootDir());
+      }
+      result = result && getTableListList()
+          .equals(other.getTableListList());
+      result = result && (hasStartTs() == other.hasStartTs());
+      if (hasStartTs()) {
+        result = result && (getStartTs()
+            == other.getStartTs());
+      }
+      result = result && (hasCompleteTs() == other.hasCompleteTs());
+      if (hasCompleteTs()) {
+        result = result && (getCompleteTs()
+            == other.getCompleteTs());
+      }
+      result = result && getAncestorsList()
+          .equals(other.getAncestorsList());
+      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 (hasBackupId()) {
+        hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER;
+        hash = (53 * hash) + getBackupId().hashCode();
+      }
+      if (hasBackupType()) {
+        hash = (37 * hash) + BACKUP_TYPE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getBackupType());
+      }
+      if (hasRootDir()) {
+        hash = (37 * hash) + ROOT_DIR_FIELD_NUMBER;
+        hash = (53 * hash) + getRootDir().hashCode();
+      }
+      if (getTableListCount() > 0) {
+        hash = (37 * hash) + TABLE_LIST_FIELD_NUMBER;
+        hash = (53 * hash) + getTableListList().hashCode();
+      }
+      if (hasStartTs()) {
+        hash = (37 * hash) + START_TS_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getStartTs());
+      }
+      if (hasCompleteTs()) {
+        hash = (37 * hash) + COMPLETE_TS_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getCompleteTs());
+      }
+      if (getAncestorsCount() > 0) {
+        hash = (37 * hash) + ANCESTORS_FIELD_NUMBER;
+        hash = (53 * hash) + getAncestorsList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage 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.BackupProtos.BackupImage parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage 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.BackupProtos.BackupImage parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage 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.BackupProtos.BackupImage parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage 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.BackupProtos.BackupImage parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage 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.BackupProtos.BackupImage 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.BackupImage}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getTableListFieldBuilder();
+          getAncestorsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        backupId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        rootDir_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        if (tableListBuilder_ == null) {
+          tableList_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000008);
+        } else {
+          tableListBuilder_.clear();
+        }
+        startTs_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        completeTs_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        if (ancestorsBuilder_ == null) {
+          ancestors_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000040);
+        } else {
+          ancestorsBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage build() {
+        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage result = new org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.backupId_ = backupId_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.backupType_ = backupType_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.rootDir_ = rootDir_;
+        if (tableListBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008)) {
+            tableList_ = java.util.Collections.unmodifiableList(tableList_);
+            bitField0_ = (bitField0_ & ~0x00000008);
+          }
+          result.tableList_ = tableList_;
+        } else {
+          result.tableList_ = tableListBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.startTs_ = startTs_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.completeTs_ = completeTs_;
+        if (ancestorsBuilder_ == null) {
+          if (((bitField0_ & 0x00000040) == 0x00000040)) {
+            ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
+            bitField0_ = (bitField0_ & ~0x00000040);
+          }
+          result.ancestors_ = ancestors_;
+        } else {
+          result.ancestors_ = ancestorsBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance()) return this;
+        if (other.hasBackupId()) {
+          bitField0_ |= 0x00000001;
+          backupId_ = other.backupId_;
+          onChanged();
+        }
+        if (other.hasBackupType()) {
+          setBackupType(other.getBackupType());
+        }
+        if (other.hasRootDir()) {
+          bitField0_ |= 0x00000004;
+          rootDir_ = other.rootDir_;
+          onChanged();
+        }
+        if (tableListBuilder_ == null) {
+          if (!other.tableList_.isEmpty()) {
+            if (tableList_.isEmpty()) {
+              tableList_ = other.tableList_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+            } else {
+              ensureTableListIsMutable();
+              tableList_.addAll(other.tableList_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.tableList_.isEmpty()) {
+            if (tableListBuilder_.isEmpty()) {
+              tableListBuilder_.dispose();
+              tableListBuilder_ = null;
+              tableList_ = other.tableList_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+              tableListBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTableListFieldBuilder() : null;
+            } else {
+              tableListBuilder_.addAllMessages(other.tableList_);
+            }
+          }
+        }
+        if (other.hasStartTs()) {
+          setStartTs(other.getStartTs());
+        }
+        if (other.hasCompleteTs()) {
+          setCompleteTs(other.getCompleteTs());
+        }
+        if (ancestorsBuilder_ == null) {
+          if (!other.ancestors_.isEmpty()) {
+            if (ancestors_.isEmpty()) {
+              ancestors_ = other.ancestors_;
+              bitField0_ = (bitField0_ & ~0x00000040);
+            } else {
+              ensureAncestorsIsMutable();
+              ancestors_.addAll(other.ancestors_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.ancestors_.isEmpty()) {
+            if (ancestorsBuilder_.isEmpty()) {
+              ancestorsBuilder_.dispose();
+              ancestorsBuilder_ = null;
+              ancestors_ = other.ancestors_;
+              bitField0_ = (bitField0_ & ~0x00000040);
+              ancestorsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getAncestorsFieldBuilder() : null;
+            } else {
+              ancestorsBuilder_.addAllMessages(other.ancestors_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasBackupId()) {
+          
+          return false;
+        }
+        if (!hasBackupType()) {
+          
+          return false;
+        }
+        if (!hasRootDir()) {
+          
+          return false;
+        }
+        if (!hasStartTs()) {
+          
+          return false;
+        }
+        if (!hasCompleteTs()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getTableListCount(); i++) {
+          if (!getTableList(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        for (int i = 0; i < getAncestorsCount(); i++) {
+          if (!getAncestors(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        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.BackupProtos.BackupImage parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string backup_id = 1;
+      private java.lang.Object backupId_ = "";
+      /**
+       * <code>required string backup_id = 1;</code>
+       */
+      public boolean hasBackupId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string backup_id = 1;</code>
+       */
+      public java.lang.String getBackupId() {
+        java.lang.Object ref = backupId_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          backupId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string backup_id = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getBackupIdBytes() {
+        java.lang.Object ref = backupId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          backupId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string backup_id = 1;</code>
+       */
+      public Builder setBackupId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        backupId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string backup_id = 1;</code>
+       */
+      public Builder clearBackupId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        backupId_ = getDefaultInstance().getBackupId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string backup_id = 1;</code>
+       */
+      public Builder setBackupIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        backupId_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required .hbase.pb.BackupType backup_type = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
+      /**
+       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+       */
+      public boolean hasBackupType() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType() {
+        return backupType_;
+      }
+      /**
+       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+       */
+      public Builder setBackupType(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000002;
+        backupType_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+       */
+      public Builder clearBackupType() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
+        onChanged();
+        return this;
+      }
+
+      // required string root_dir = 3;
+      private java.lang.Object rootDir_ = "";
+      /**
+       * <code>required string root_dir = 3;</code>
+       */
+      public boolean hasRootDir() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required string root_dir = 3;</code>
+       */
+      public java.lang.String getRootDir() {
+        java.lang.Object ref = rootDir_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          rootDir_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string root_dir = 3;</code>
+       */
+      public com.google.protobuf.ByteString
+          getRootDirBytes() {
+        java.lang.Object ref = rootDir_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          rootDir_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string root_dir = 3;</code>
+       */
+      public Builder setRootDir(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        rootDir_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string root_dir = 3;</code>
+       */
+      public Builder clearRootDir() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        rootDir_ = getDefaultInstance().getRootDir();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string root_dir = 3;</code>
+       */
+      public Builder setRootDirBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        rootDir_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated .hbase.pb.TableName table_list = 4;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableList_ =
+        java.util.Collections.emptyList();
+      private void ensureTableListIsMutable() {
+        if (!((bitField0_ & 0x00000008) == 0x00000008)) {
+          tableList_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>(tableList_);
+          bitField0_ |= 0x00000008;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableListBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableListList() {
+        if (tableListBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(tableList_);
+        } else {
+          return tableListBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public int getTableListCount() {
+        if (tableListBuilder_ == null) {
+          return tableList_.size();
+        } else {
+          return tableListBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index) {
+        if (tableListBuilder_ == null) {
+          return tableList_.get(index);
+        } else {
+          return tableListBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder setTableList(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableListBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableListIsMutable();
+          tableList_.set(index, value);
+          onChanged();
+        } else {
+          tableListBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder setTableList(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableListBuilder_ == null) {
+          ensureTableListIsMutable();
+          tableList_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableListBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder addTableList(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableListBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableListIsMutable();
+          tableList_.add(value);
+          onChanged();
+        } else {
+          tableListBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder addTableList(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableListBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableListIsMutable();
+          tableList_.add(index, value);
+          onChanged();
+        } else {
+          tableListBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder addTableList(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableListBuilder_ == null) {
+          ensureTableListIsMutable();
+          tableList_.add(builderForValue.build());
+          onChanged();
+        } else {
+          tableListBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder addTableList(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableListBuilder_ == null) {
+          ensureTableListIsMutable();
+          tableList_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableListBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder addAllTableList(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> values) {
+        if (tableListBuilder_ == null) {
+          ensureTableListIsMutable();
+          super.addAll(values, tableList_);
+          onChanged();
+        } else {
+          tableListBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder clearTableList() {
+        if (tableListBuilder_ == null) {
+          tableList_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000008);
+          onChanged();
+        } else {
+          tableListBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public Builder removeTableList(int index) {
+        if (tableListBuilder_ == null) {
+          ensureTableListIsMutable();
+          tableList_.remove(index);
+          onChanged();
+        } else {
+          tableListBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableListBuilder(
+          int index) {
+        return getTableListFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
+          int index) {
+        if (tableListBuilder_ == null) {
+          return tableList_.get(index);  } else {
+          return tableListBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+           getTableListOrBuilderList() {
+        if (tableListBuilder_ != null) {
+          return tableListBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(tableList_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTableListBuilder() {
+        return getTableListFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTableListBuilder(
+          int index) {
+        return getTableListFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder> 
+           getTableListBuilderList() {
+        return getTableListFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableListFieldBuilder() {
+        if (tableListBuilder_ == null) {
+          tableListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableList_,
+                  ((bitField0_ & 0x00000008) == 0x00000008),
+                  getParentForChildren(),
+                  isClean());
+          tableList_ = null;
+        }
+        return tableListBuilder_;
+      }
+
+      // required uint64 start_ts = 5;
+      private long startTs_ ;
+      /**
+       * <code>required uint64 start_ts = 5;</code>
+       */
+      public boolean hasStartTs() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>required uint64 start_ts = 5;</code>
+       */
+      public long getStartTs() {
+        return startTs_;
+      }
+      /**
+       * <code>required uint64 start_ts = 5;</code>
+       */
+      public Builder setStartTs(long value) {
+        bitField0_ |= 0x00000010;
+        startTs_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 start_ts = 5;</code>
+       */
+      public Builder clearStartTs() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        startTs_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 complete_ts = 6;
+      private long completeTs_ ;
+      /**
+       * <code>required uint64 complete_ts = 6;</code>
+       */
+      public boolean hasCompleteTs() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>required uint64 complete_ts = 6;</code>
+       */
+      public long getCompleteTs() {
+        return completeTs_;
+      }
+      /**
+       * <code>required uint64 complete_ts = 6;</code>
+       */
+      public Builder setCompleteTs(long value) {
+        bitField0_ |= 0x00000020;
+        completeTs_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 complete_ts = 6;</code>
+       */
+      public Builder clearCompleteTs() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        completeTs_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // repeated .hbase.pb.BackupImage ancestors = 7;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> ancestors_ =
+        java.util.Collections.emptyList();
+      private void ensureAncestorsIsMutable() {
+        if (!((bitField0_ & 0x00000040) == 0x00000040)) {
+          ancestors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage>(ancestors_);
+          bitField0_ |= 0x00000040;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> ancestorsBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> getAncestorsList() {
+        if (ancestorsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(ancestors_);
+        } else {
+          return ancestorsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public int getAncestorsCount() {
+        if (ancestorsBuilder_ == null) {
+          return ancestors_.size();
+        } else {
+          return ancestorsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) {
+        if (ancestorsBuilder_ == null) {
+          return ancestors_.get(index);
+        } else {
+          return ancestorsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder setAncestors(
+          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage value) {
+        if (ancestorsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAncestorsIsMutable();
+          ancestors_.set(index, value);
+          onChanged();
+        } else {
+          ancestorsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder setAncestors(
+          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) {
+        if (ancestorsBuilder_ == null) {
+          ensureAncestorsIsMutable();
+          ancestors_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          ancestorsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder addAncestors(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage value) {
+        if (ancestorsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAncestorsIsMutable();
+          ancestors_.add(value);
+          onChanged();
+        } else {
+          ancestorsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder addAncestors(
+          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage value) {
+        if (ancestorsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureAncestorsIsMutable();
+          ancestors_.add(index, value);
+          onChanged();
+        } else {
+          ancestorsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder addAncestors(
+          org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) {
+        if (ancestorsBuilder_ == null) {
+          ensureAncestorsIsMutable();
+          ancestors_.add(builderForValue.build());
+          onChanged();
+        } else {
+          ancestorsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder addAncestors(
+          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) {
+        if (ancestorsBuilder_ == null) {
+          ensureAncestorsIsMutable();
+          ancestors_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          ancestorsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder addAllAncestors(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> values) {
+        if (ancestorsBuilder_ == null) {
+          ensureAncestorsIsMutable();
+          super.addAll(values, ancestors_);
+          onChanged();
+        } else {
+          ancestorsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder clearAncestors() {
+        if (ancestorsBuilder_ == null) {
+          ancestors_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000040);
+          onChanged();
+        } else {
+          ancestorsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public Builder removeAncestors(int index) {
+        if (ancestorsBuilder_ == null) {
+          ensureAncestorsIsMutable();
+          ancestors_.remove(index);
+          onChanged();
+        } else {
+          ancestorsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder getAncestorsBuilder(
+          int index) {
+        return getAncestorsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
+          int index) {
+        if (ancestorsBuilder_ == null) {
+          return ancestors_.get(index);  } else {
+          return ancestorsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
+           getAncestorsOrBuilderList() {
+        if (ancestorsBuilder_ != null) {
+          return ancestorsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(ancestors_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder addAncestorsBuilder() {
+        return getAncestorsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder addAncestorsBuilder(
+          int index) {
+        return getAncestorsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder> 
+           getAncestorsBuilderList() {
+        return getAncestorsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
+          getAncestorsFieldBuilder() {
+        if (ancestorsBuilder_ == null) {
+          ancestorsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder>(
+                  ancestors_,
+                  ((bitField0_ & 0x00000040) == 0x00000040),
+                  getParentForChildren(),
+                  isClean());
+          ancestors_ = null;
+        }
+        return ancestorsBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.BackupImage)
+    }
+
+    static {
+      defaultInstance = new BackupImage(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.BackupImage)
+  }
+
+  public interface ServerTimestampOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string server = 1;
+    /**
+     * <code>required string server = 1;</code>
+     */
+    boolean hasServer();
+    /**
+     * <code>required string server = 1;</code>
+     */
+    java.lang.String getServer();
+    /**
+     * <code>required string server = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getServerBytes();
+
+    // required uint64 timestamp = 2;
+    /**
+     * <code>required uint64 timestamp = 2;</code>
+     */
+    boolean hasTimestamp();
+    /**
+     * <code>required uint64 timestamp = 2;</code>
+     */
+    long getTimestamp();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.ServerTimestamp}
+   */
+  public static final class ServerTimestamp extends
+      com.google.protobuf.GeneratedMessage
+      implements ServerTimestampOrBuilder {
+    // Use ServerTimestamp.newBuilder() to construct.
+    private ServerTimestamp(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ServerTimestamp(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ServerTimestamp defaultInstance;
+    public static ServerTimestamp getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ServerTimestamp getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ServerTimestamp(
+        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 10: {
+              bitField0_ |= 0x00000001;
+              server_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              timestamp_ = input.readUInt64();
+              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.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ServerTimestamp> PARSER =
+        new com.google.protobuf.AbstractParser<ServerTimestamp>() {
+      public ServerTimestamp parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ServerTimestamp(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ServerTimestamp> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string server = 1;
+    public static final int SERVER_FIELD_NUMBER = 1;
+    private java.lang.Object server_;
+    /**
+     * <code>required string server = 1;</code>
+     */
+    public boolean hasServer() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string server = 1;</code>
+     */
+    public java.lang.String getServer() {
+      java.lang.Object ref = server_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          server_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string server = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getServerBytes() {
+      java.lang.Object ref = server_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        server_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required uint64 timestamp = 2;
+    public static final int TIMESTAMP_FIELD_NUMBER = 2;
+    private long timestamp_;
+    /**
+     * <code>required uint64 timestamp = 2;</code>
+     */
+    public boolean hasTimestamp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required uint64 timestamp = 2;</code>
+     */
+    public long getTimestamp() {
+      return timestamp_;
+    }
+
+    private void initFields() {
+      server_ = "";
+      timestamp_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasServer()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTimestamp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getServerBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, timestamp_);
+      }
+      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
+          .computeBytesSize(1, getServerBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, timestamp_);
+      }
+      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.BackupProtos.ServerTimestamp)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp other = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp) obj;
+
+      boolean result = true;
+      result = result && (hasServer() == other.hasServer());
+      if (hasServer()) {
+        result = result && getServer()
+            .equals(other.getServer());
+      }
+      result = result && (hasTimestamp() == other.hasTimestamp());
+      if (hasTimestamp()) {
+        result = result && (getTimestamp()
+            == other.getTimestamp());
+      }
+      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 (hasServer()) {
+        hash = (37 * hash) + SERVER_FIELD_NUMBER;
+        hash = (53 * hash) + getServer().hashCode();
+      }
+      if (hasTimestamp()) {
+        hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimestamp());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp 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.BackupProtos.ServerTimestamp parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp 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.BackupProtos.ServerTimestamp parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp 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.BackupProtos.ServerTimestamp parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp 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.BackupProtos.ServerTimestamp parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp 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.BackupProtos.ServerTimestamp 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.ServerTimestamp}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestampOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.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();
+        server_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        timestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp build() {
+        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp result = new org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.server_ = server_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.timestamp_ = timestamp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance()) return this;
+        if (other.hasServer()) {
+          bitField0_ |= 0x00000001;
+          server_ = other.server_;
+          onChanged();
+        }
+        if (other.hasTimestamp()) {
+          setTimestamp(other.getTimestamp());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasServer()) {
+          
+          return false;
+        }
+        if (!hasTimestamp()) {
+          
+          return false;
+        }
+        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.BackupProtos.ServerTimestamp parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string server = 1;
+      private java.lang.Object server_ = "";
+      /**
+       * <code>required string server = 1;</code>
+       */
+      public boolean hasServer() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string server = 1;</code>
+       */
+      public java.lang.String getServer() {
+        java.lang.Object ref = server_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          server_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string server = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getServerBytes() {
+        java.lang.Object ref = server_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          server_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string server = 1;</code>
+       */
+      public Builder setServer(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        server_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string server = 1;</code>
+       */
+      public Builder clearServer() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        server_ = getDefaultInstance().getServer();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string server = 1;</code>
+       */
+      public Builder setServerBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        server_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 timestamp = 2;
+      private long timestamp_ ;
+      /**
+       * <code>required uint64 timestamp = 2;</code>
+       */
+      public boolean hasTimestamp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required uint64 timestamp = 2;</code>
+       */
+      public long getTimestamp() {
+        return timestamp_;
+      }
+      /**
+       * <code>required uint64 timestamp = 2;</code>
+       */
+      public Builder setTimestamp(long value) {
+        bitField0_ |= 0x00000002;
+        timestamp_ = value;
+        onC

<TRUNCATED>

[45/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
new file mode 100644
index 0000000..14235ce
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
@@ -0,0 +1,297 @@
+/**
+ * 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.backup.mapreduce;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.util.Arrays;
+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.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupContext;
+import org.apache.hadoop.hbase.backup.impl.BackupCopyService;
+import org.apache.hadoop.hbase.backup.impl.BackupHandler;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.impl.BackupUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.tools.DistCp;
+import org.apache.hadoop.tools.DistCpConstants;
+import org.apache.hadoop.tools.DistCpOptions;
+/**
+ * Copier for backup operation. Basically, there are 2 types of copy. One is copying from snapshot,
+ * which bases on extending ExportSnapshot's function with copy progress reporting to ZooKeeper
+ * implementation. The other is copying for incremental log files, which bases on extending
+ * DistCp's function with copy progress reporting to ZooKeeper implementation.
+ *
+ * For now this is only a wrapper. The other features such as progress and increment backup will be
+ * implemented in future jira
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MapReduceBackupCopyService implements BackupCopyService {
+  private static final Log LOG = LogFactory.getLog(MapReduceBackupCopyService.class);
+
+  private Configuration conf;
+  // private static final long BYTES_PER_MAP = 2 * 256 * 1024 * 1024;
+
+  // Accumulated progress within the whole backup process for the copy operation
+  private float progressDone = 0.1f;
+  private long bytesCopied = 0;
+  private static float INIT_PROGRESS = 0.1f;
+
+  // The percentage of the current copy task within the whole task if multiple time copies are
+  // needed. The default value is 100%, which means only 1 copy task for the whole.
+  private float subTaskPercntgInWholeTask = 1f;
+
+  public MapReduceBackupCopyService() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Get the current copy task percentage within the whole task if multiple copies are needed.
+   * @return the current copy task percentage
+   */
+  public float getSubTaskPercntgInWholeTask() {
+    return subTaskPercntgInWholeTask;
+  }
+
+  /**
+   * Set the current copy task percentage within the whole task if multiple copies are needed. Must
+   * be called before calling
+   * {@link #copy(BackupHandler, Configuration, Type, String[])}
+   * @param subTaskPercntgInWholeTask The percentage of the copy subtask
+   */
+  public void setSubTaskPercntgInWholeTask(float subTaskPercntgInWholeTask) {
+    this.subTaskPercntgInWholeTask = subTaskPercntgInWholeTask;
+  }
+
+  class SnapshotCopy extends ExportSnapshot {
+    private BackupContext backupContext;
+    private TableName table;
+
+    public SnapshotCopy(BackupContext backupContext, TableName table) {
+      super();
+      this.backupContext = backupContext;
+      this.table = table;
+    }
+
+    public TableName getTable() {
+      return this.table;
+    }
+  }
+
+  // Extends DistCp for progress updating to hbase:backup
+  // during backup. Using DistCpV2 (MAPREDUCE-2765).
+  // Simply extend it and override execute() method to get the
+  // Job reference for progress updating.
+  // Only the argument "src1, [src2, [...]] dst" is supported,
+  // no more DistCp options.
+  class BackupDistCp extends DistCp {
+
+    private BackupContext backupContext;
+    private BackupManager backupManager;
+
+    public BackupDistCp(Configuration conf, DistCpOptions options, BackupContext backupContext,
+        BackupManager backupManager)
+        throws Exception {
+      super(conf, options);
+      this.backupContext = backupContext;
+      this.backupManager = backupManager;
+    }
+
+    @Override
+    public Job execute() throws Exception {
+
+      // reflection preparation for private methods and fields
+      Class<?> classDistCp = org.apache.hadoop.tools.DistCp.class;
+      Method methodCreateMetaFolderPath = classDistCp.getDeclaredMethod("createMetaFolderPath");
+      Method methodCreateJob = classDistCp.getDeclaredMethod("createJob");
+      Method methodCreateInputFileListing =
+          classDistCp.getDeclaredMethod("createInputFileListing", Job.class);
+      Method methodCleanup = classDistCp.getDeclaredMethod("cleanup");
+
+      Field fieldInputOptions = classDistCp.getDeclaredField("inputOptions");
+      Field fieldMetaFolder = classDistCp.getDeclaredField("metaFolder");
+      Field fieldJobFS = classDistCp.getDeclaredField("jobFS");
+      Field fieldSubmitted = classDistCp.getDeclaredField("submitted");
+
+      methodCreateMetaFolderPath.setAccessible(true);
+      methodCreateJob.setAccessible(true);
+      methodCreateInputFileListing.setAccessible(true);
+      methodCleanup.setAccessible(true);
+
+      fieldInputOptions.setAccessible(true);
+      fieldMetaFolder.setAccessible(true);
+      fieldJobFS.setAccessible(true);
+      fieldSubmitted.setAccessible(true);
+
+      // execute() logic starts here
+      assert fieldInputOptions.get(this) != null;
+      assert getConf() != null;
+
+      Job job = null;
+      try {
+        synchronized (this) {
+          // Don't cleanup while we are setting up.
+          fieldMetaFolder.set(this, methodCreateMetaFolderPath.invoke(this));
+          fieldJobFS.set(this, ((Path) fieldMetaFolder.get(this)).getFileSystem(getConf()));
+
+          job = (Job) methodCreateJob.invoke(this);
+        }
+        methodCreateInputFileListing.invoke(this, job);
+
+        // Get the total length of the source files
+        List<Path> srcs = ((DistCpOptions) fieldInputOptions.get(this)).getSourcePaths();
+        long totalSrcLgth = 0;
+        for (Path aSrc : srcs) {
+          totalSrcLgth += BackupUtil.getFilesLength(aSrc.getFileSystem(getConf()), aSrc);
+        }
+
+        // submit the copy job
+        job.submit();
+        fieldSubmitted.set(this, true);
+
+        // after submit the MR job, set its handler in backup handler for cancel process
+        // this.backupHandler.copyJob = job;
+
+        // Update the copy progress to ZK every 0.5s if progress value changed
+        int progressReportFreq =
+            this.getConf().getInt("hbase.backup.progressreport.frequency", 500);
+        float lastProgress = progressDone;
+        while (!job.isComplete()) {
+          float newProgress =
+              progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS);
+
+          if (newProgress > lastProgress) {
+
+            BigDecimal progressData =
+                new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP);
+            String newProgressStr = progressData + "%";
+            LOG.info("Progress: " + newProgressStr);
+            BackupHandler.updateProgress(backupContext, backupManager, progressData.intValue(),
+              bytesCopied);
+            LOG.debug("Backup progress data updated to hbase:backup: \"Progress: " + newProgressStr
+              + ".\"");
+            lastProgress = newProgress;
+          }
+          Thread.sleep(progressReportFreq);
+        }
+
+        // update the progress data after copy job complete
+        float newProgress =
+            progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS);
+        BigDecimal progressData =
+            new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP);
+
+        String newProgressStr = progressData + "%";
+        LOG.info("Progress: " + newProgressStr);
+
+        // accumulate the overall backup progress
+        progressDone = newProgress;
+        bytesCopied += totalSrcLgth;
+
+        BackupHandler.updateProgress(backupContext, backupManager, progressData.intValue(),
+          bytesCopied);
+        LOG.debug("Backup progress data updated to hbase:backup: \"Progress: " + newProgressStr
+          + " - " + bytesCopied + " bytes copied.\"");
+
+      } finally {
+        if (!fieldSubmitted.getBoolean(this)) {
+          methodCleanup.invoke(this);
+        }
+      }
+
+      String jobID = job.getJobID().toString();
+      job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID, jobID);
+
+      LOG.debug("DistCp job-id: " + jobID);
+      return job;
+    }
+
+  }
+
+  /**
+   * Do backup copy based on different types.
+   * @param context The backup context
+   * @param conf The hadoop configuration
+   * @param copyType The backup copy type
+   * @param options Options for customized ExportSnapshot or DistCp
+   * @throws Exception exception
+   */
+  @Override
+  public int copy(BackupContext context, BackupManager backupManager, Configuration conf,
+      BackupCopyService.Type copyType, String[] options) throws IOException {
+    int res = 0;
+
+    try {
+      if (copyType == Type.FULL) {
+        SnapshotCopy snapshotCp =
+            new SnapshotCopy(context, context.getTableBySnapshot(options[1]));
+        LOG.debug("Doing SNAPSHOT_COPY");
+        // Make a new instance of conf to be used by the snapshot copy class.
+        snapshotCp.setConf(new Configuration(conf));
+        res = snapshotCp.run(options);
+      } else if (copyType == Type.INCREMENTAL) {
+        LOG.debug("Doing COPY_TYPE_DISTCP");
+        setSubTaskPercntgInWholeTask(1f);
+
+        BackupDistCp distcp = new BackupDistCp(new Configuration(conf), null, context,
+          backupManager);
+        // Handle a special case where the source file is a single file.
+        // In this case, distcp will not create the target dir. It just take the
+        // target as a file name and copy source file to the target (as a file name).
+        // We need to create the target dir before run distcp.
+        LOG.debug("DistCp options: " + Arrays.toString(options));
+        if (options.length == 2) {
+          Path dest = new Path(options[1]);
+          FileSystem destfs = dest.getFileSystem(conf);
+          if (!destfs.exists(dest)) {
+            destfs.mkdirs(dest);
+          }
+        }
+
+        res = distcp.run(options);
+      }
+      return res;
+
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
new file mode 100644
index 0000000..203c9a3
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
@@ -0,0 +1,74 @@
+/**
+ * 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.backup.mapreduce;
+
+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.TableName;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupUtil;
+import org.apache.hadoop.hbase.backup.impl.IncrementalRestoreService;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.mapreduce.WALPlayer;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MapReduceRestoreService implements IncrementalRestoreService {
+  public static final Log LOG = LogFactory.getLog(MapReduceRestoreService.class);
+
+  private WALPlayer player;
+
+  public MapReduceRestoreService() {
+    this.player = new WALPlayer();
+  }
+
+  @Override
+  public void run(String logDir, TableName[] tableNames, TableName[] newTableNames) throws IOException {
+    String tableStr = BackupUtil.join(tableNames);
+    String newTableStr = BackupUtil.join(newTableNames);
+
+    // WALPlayer reads all files in arbitrary directory structure and creates a Map task for each
+    // log file
+
+    String[] playerArgs = { logDir, tableStr, newTableStr };
+    LOG.info("Restore incremental backup from directory " + logDir + " from hbase tables "
+        + BackupUtil.join(tableNames) + " to tables "
+        + BackupUtil.join(newTableNames));
+    try {
+      player.run(playerArgs);
+    } catch (Exception e) {
+      throw new IOException("cannot restore from backup directory " + logDir
+        + " (check Hadoop and HBase logs) " + e);
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return player.getConf();
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.player.setConf(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
new file mode 100644
index 0000000..dae24a6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
@@ -0,0 +1,119 @@
+/**
+ *
+ * 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.backup.master;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.FileStatus;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
+
+/**
+ * Implementation of a log cleaner that checks if a log is still scheduled for
+ * incremental backup before deleting it when its TTL is over.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BackupLogCleaner extends BaseLogCleanerDelegate {
+  private static final Log LOG = LogFactory.getLog(BackupLogCleaner.class);
+
+  private boolean stopped = false;
+
+  public BackupLogCleaner() {
+  }
+
+  @Override
+  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
+    // all members of this class are null if backup is disabled,
+    // so we cannot filter the files
+    if (this.getConf() == null) {
+      return files;
+    }
+    
+    List<FileStatus> list = new ArrayList<FileStatus>();
+    // TODO: LogCleaners do not have a way to get the Connection from Master. We should find a
+    // way to pass it down here, so that this connection is not re-created every time.
+    // It is expensive
+    try(Connection connection = ConnectionFactory.createConnection(this.getConf());
+        final BackupSystemTable table = new BackupSystemTable(connection)) {
+
+      // If we do not have recorded backup sessions
+      if (!table.hasBackupSessions()) {
+        return files;
+      }
+      
+      for(FileStatus file: files){
+        String wal = file.getPath().toString();
+        boolean logInSystemTable = table.checkWALFile(wal);
+        if(LOG.isDebugEnabled()) {
+          if(logInSystemTable) {
+            LOG.debug("Found log file in hbase:backup, deleting: " + wal);
+            list.add(file);
+          } else {
+            LOG.debug("Didn't find this log in hbase:backup, keeping: " + wal);
+          }
+        }
+      }
+      return list;  
+    } catch (IOException e) {
+      LOG.error("Failed to get hbase:backup table, therefore will keep all files", e);
+      // nothing to delete
+      return new ArrayList<FileStatus>();
+    }
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    // If backup is disabled, keep all members null
+    if (!config.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
+      LOG.warn("Backup is disabled - allowing all wals to be deleted");
+      return;
+    }
+    super.setConf(config);
+  }
+
+  @Override
+  public void stop(String why) {
+    if (this.stopped) {
+      return;
+    }
+    this.stopped = true;
+    LOG.info("Stopping BackupLogCleaner");
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
new file mode 100644
index 0000000..f96682f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
@@ -0,0 +1,129 @@
+/**
+ * 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.backup.master;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.MetricsMaster;
+import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
+import org.apache.hadoop.hbase.procedure.Procedure;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
+import org.apache.zookeeper.KeeperException;
+
+public class LogRollMasterProcedureManager extends MasterProcedureManager {
+
+  public static final String ROLLLOG_PROCEDURE_SIGNATURE = "rolllog-proc";
+  public static final String ROLLLOG_PROCEDURE_NAME = "rolllog";
+  private static final Log LOG = LogFactory.getLog(LogRollMasterProcedureManager.class);
+
+  private MasterServices master;
+  private ProcedureCoordinator coordinator;
+  private boolean done;
+
+  @Override
+  public void stop(String why) {
+    LOG.info("stop: " + why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return false;
+  }
+
+  @Override
+  public void initialize(MasterServices master, MetricsMaster metricsMaster)
+      throws KeeperException, IOException, UnsupportedOperationException {
+    this.master = master;
+    this.done = false;
+
+    // setup the default procedure coordinator
+    String name = master.getServerName().toString();
+    ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, 1);
+    BaseCoordinatedStateManager coordManager =
+        (BaseCoordinatedStateManager) CoordinatedStateManagerFactory
+        .getCoordinatedStateManager(master.getConfiguration());
+    coordManager.initialize(master);
+
+    ProcedureCoordinatorRpcs comms =
+        coordManager.getProcedureCoordinatorRpcs(getProcedureSignature(), name);
+
+    this.coordinator = new ProcedureCoordinator(comms, tpool);
+  }
+
+  @Override
+  public String getProcedureSignature() {
+    return ROLLLOG_PROCEDURE_SIGNATURE;
+  }
+
+  @Override
+  public void execProcedure(ProcedureDescription desc) throws IOException {
+    this.done = false;
+    // start the process on the RS
+    ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getInstance());
+    List<ServerName> serverNames = master.getServerManager().getOnlineServersList();
+    List<String> servers = new ArrayList<String>();
+    for (ServerName sn : serverNames) {
+      servers.add(sn.toString());
+    }
+    Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), new byte[0], servers);
+    if (proc == null) {
+      String msg = "Failed to submit distributed procedure for '" + desc.getInstance() + "'";
+      LOG.error(msg);
+      throw new IOException(msg);
+    }
+
+    try {
+      // wait for the procedure to complete. A timer thread is kicked off that should cancel this
+      // if it takes too long.
+      proc.waitForCompleted();
+      LOG.info("Done waiting - exec procedure for " + desc.getInstance());
+      LOG.info("Distributed roll log procedure is successful!");
+      this.done = true;
+    } catch (InterruptedException e) {
+      ForeignException ee =
+          new ForeignException("Interrupted while waiting for roll log procdure to finish", e);
+      monitor.receive(ee);
+      Thread.currentThread().interrupt();
+    } catch (ForeignException e) {
+      ForeignException ee =
+          new ForeignException("Exception while waiting for roll log procdure to finish", e);
+      monitor.receive(ee);
+    }
+    monitor.rethrowException();
+  }
+
+  @Override
+  public boolean isProcedureDone(ProcedureDescription desc) throws IOException {
+    return done;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
new file mode 100644
index 0000000..d524140
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
@@ -0,0 +1,143 @@
+/**
+ * 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.backup.regionserver;
+
+import java.util.HashMap;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.procedure.ProcedureMember;
+import org.apache.hadoop.hbase.procedure.Subprocedure;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
+
+/**
+ * This backup subprocedure implementation forces a log roll on the RS.
+ */
+public class LogRollBackupSubprocedure extends Subprocedure {
+  private static final Log LOG = LogFactory.getLog(LogRollBackupSubprocedure.class);
+
+  private final RegionServerServices rss;
+  private final LogRollBackupSubprocedurePool taskManager;
+  private FSHLog hlog;
+
+  public LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember member,
+      ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
+      LogRollBackupSubprocedurePool taskManager) {
+
+    super(member, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, errorListener,
+      wakeFrequency, timeout);
+    LOG.info("Constructing a LogRollBackupSubprocedure.");
+    this.rss = rss;
+    this.taskManager = taskManager;
+  }
+
+  /**
+   * Callable task. TODO. We don't need a thread pool to execute roll log. This can be simplified
+   * with no use of subprocedurepool.
+   */
+  class RSRollLogTask implements Callable<Void> {
+    RSRollLogTask() {
+    }
+
+    @Override
+    public Void call() throws Exception {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("++ DRPC started: " + rss.getServerName());
+      }
+      hlog = (FSHLog) rss.getWAL(null);
+      long filenum = hlog.getFilenum();
+
+      LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum);
+      hlog.rollWriter(true);
+      LOG.info("After roll log in backup subprocedure, current log number: " + hlog.getFilenum());
+
+      Connection connection = rss.getConnection();
+      try(final BackupSystemTable table = new BackupSystemTable(connection)) {
+        // sanity check, good for testing
+        HashMap<String, Long> serverTimestampMap = table.readRegionServerLastLogRollResult();
+        String host = rss.getServerName().getHostname();
+        int port = rss.getServerName().getPort();
+        String server = host + ":" + port;
+        Long sts = serverTimestampMap.get(host);
+        if (sts != null && sts > filenum) {
+          LOG.warn("Won't update server's last roll log result: current="
+              + sts + " new=" + filenum);
+          return null;
+        }
+        // write the log number to hbase:backup.
+        table.writeRegionServerLastLogRollResult(server, filenum);
+        return null;
+      } catch (Exception e) {
+        LOG.error(e);
+        throw e; // TODO: is this correct?
+      }
+    }
+  }
+
+  private void rolllog() throws ForeignException {
+    monitor.rethrowException();
+
+    taskManager.submitTask(new RSRollLogTask());
+    monitor.rethrowException();
+
+    // wait for everything to complete.
+    taskManager.waitForOutstandingTasks();
+    monitor.rethrowException();
+
+  }
+
+  @Override
+  public void acquireBarrier() throws ForeignException {
+    // do nothing, executing in inside barrier step.
+  }
+
+  /**
+   * do a log roll.
+   * @return some bytes
+   */
+  @Override
+  public byte[] insideBarrier() throws ForeignException {
+    rolllog();
+    // FIXME
+    return null;
+  }
+
+  /**
+   * Cancel threads if they haven't finished.
+   */
+  @Override
+  public void cleanup(Exception e) {
+    taskManager.abort("Aborting log roll subprocedure tasks for backup due to error", e);
+  }
+
+  /**
+   * Hooray!
+   */
+  public void releaseBarrier() {
+    // NO OP
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
new file mode 100644
index 0000000..1ca638c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
@@ -0,0 +1,137 @@
+/**
+ * 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.backup.regionserver;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+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.hbase.Abortable;
+import org.apache.hadoop.hbase.DaemonThreadFactory;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+
+/**
+ * Handle running each of the individual tasks for completing a backup procedure
+ * on a regionserver.
+ */
+public class LogRollBackupSubprocedurePool implements Closeable, Abortable {
+  private static final Log LOG = LogFactory.getLog(LogRollBackupSubprocedurePool.class);
+
+  /** Maximum number of concurrent snapshot region tasks that can run concurrently */
+  private static final String CONCURENT_BACKUP_TASKS_KEY = "hbase.backup.region.concurrentTasks";
+  private static final int DEFAULT_CONCURRENT_BACKUP_TASKS = 3;
+
+  private final ExecutorCompletionService<Void> taskPool;
+  private final ThreadPoolExecutor executor;
+  private volatile boolean aborted;
+  private final List<Future<Void>> futures = new ArrayList<Future<Void>>();
+  private final String name;
+
+  public LogRollBackupSubprocedurePool(String name, Configuration conf) {
+    // configure the executor service
+    long keepAlive =
+        conf.getLong(LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_KEY,
+          LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
+    int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
+    this.name = name;
+    executor =
+        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
+          new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs(" + name
+            + ")-backup-pool"));
+    taskPool = new ExecutorCompletionService<Void>(executor);
+  }
+
+  /**
+   * Submit a task to the pool.
+   */
+  public void submitTask(final Callable<Void> task) {
+    Future<Void> f = this.taskPool.submit(task);
+    futures.add(f);
+  }
+
+  /**
+   * Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)}
+   * @return <tt>true</tt> on success, <tt>false</tt> otherwise
+   * @throws ForeignException exception
+   */
+  public boolean waitForOutstandingTasks() throws ForeignException {
+    LOG.debug("Waiting for backup procedure to finish.");
+
+    try {
+      for (Future<Void> f : futures) {
+        f.get();
+      }
+      return true;
+    } catch (InterruptedException e) {
+      if (aborted) {
+        throw new ForeignException("Interrupted and found to be aborted while waiting for tasks!",
+            e);
+      }
+      Thread.currentThread().interrupt();
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof ForeignException) {
+        throw (ForeignException) e.getCause();
+      }
+      throw new ForeignException(name, e.getCause());
+    } finally {
+      // close off remaining tasks
+      for (Future<Void> f : futures) {
+        if (!f.isDone()) {
+          f.cancel(true);
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Attempt to cleanly shutdown any running tasks - allows currently running tasks to cleanly
+   * finish
+   */
+  @Override
+  public void close() {
+    executor.shutdown();
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    if (this.aborted) {
+      return;
+    }
+
+    this.aborted = true;
+    LOG.warn("Aborting because: " + why, e);
+    this.executor.shutdownNow();
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.aborted;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
new file mode 100644
index 0000000..aca190c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
@@ -0,0 +1,168 @@
+/**
+ * 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.backup.regionserver;
+
+
+import java.io.IOException;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.procedure.ProcedureMember;
+import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
+import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
+import org.apache.hadoop.hbase.procedure.Subprocedure;
+import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * This manager class handles the work dealing with backup for a {@link HRegionServer}.
+ * <p>
+ * This provides the mechanism necessary to kick off a backup specific {@link Subprocedure} that is
+ * responsible by this region server. If any failures occur with the subprocedure, the manager's
+ * procedure member notifies the procedure coordinator to abort all others.
+ * <p>
+ * On startup, requires {@link #start()} to be called.
+ * <p>
+ * On shutdown, requires org.apache.hadoop.hbase.procedure.ProcedureMember.close() to be
+ * called
+ */
+public class LogRollRegionServerProcedureManager extends RegionServerProcedureManager {
+
+  private static final Log LOG = LogFactory.getLog(LogRollRegionServerProcedureManager.class);
+
+  /** Conf key for number of request threads to start backup on regionservers */
+  public static final String BACKUP_REQUEST_THREADS_KEY = "hbase.backup.region.pool.threads";
+  /** # of threads for backup work on the rs. */
+  public static final int BACKUP_REQUEST_THREADS_DEFAULT = 10;
+
+  public static final String BACKUP_TIMEOUT_MILLIS_KEY = "hbase.backup.timeout";
+  public static final long BACKUP_TIMEOUT_MILLIS_DEFAULT = 60000;
+
+  /** Conf key for millis between checks to see if backup work completed or if there are errors */
+  public static final String BACKUP_REQUEST_WAKE_MILLIS_KEY = "hbase.backup.region.wakefrequency";
+  /** Default amount of time to check for errors while regions finish backup work */
+  private static final long BACKUP_REQUEST_WAKE_MILLIS_DEFAULT = 500;
+
+  private RegionServerServices rss;
+  private ProcedureMemberRpcs memberRpcs;
+  private ProcedureMember member;
+
+  /**
+   * Create a default backup procedure manager
+   */
+  public LogRollRegionServerProcedureManager() {
+  }
+
+  /**
+   * Start accepting backup procedure requests.
+   */
+  @Override
+  public void start() {
+    this.memberRpcs.start(rss.getServerName().toString(), member);
+    LOG.info("Started region server backup manager.");
+  }
+
+  /**
+   * Close <tt>this</tt> and all running backup procedure tasks
+   * @param force forcefully stop all running tasks
+   * @throws IOException exception
+   */
+  @Override
+  public void stop(boolean force) throws IOException {
+    String mode = force ? "abruptly" : "gracefully";
+    LOG.info("Stopping RegionServerBackupManager " + mode + ".");
+
+    try {
+      this.member.close();
+    } finally {
+      this.memberRpcs.close();
+    }
+  }
+
+  /**
+   * If in a running state, creates the specified subprocedure for handling a backup procedure.
+   * @return Subprocedure to submit to the ProcedureMemeber.
+   */
+  public Subprocedure buildSubprocedure() {
+
+    // don't run a backup if the parent is stop(ping)
+    if (rss.isStopping() || rss.isStopped()) {
+      throw new IllegalStateException("Can't start backup procedure on RS: " + rss.getServerName()
+        + ", because stopping/stopped!");
+    }
+
+    LOG.info("Attempting to run a roll log procedure for backup.");
+    ForeignExceptionDispatcher errorDispatcher = new ForeignExceptionDispatcher();
+    Configuration conf = rss.getConfiguration();
+    long timeoutMillis = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT);
+    long wakeMillis =
+        conf.getLong(BACKUP_REQUEST_WAKE_MILLIS_KEY, BACKUP_REQUEST_WAKE_MILLIS_DEFAULT);
+
+    LogRollBackupSubprocedurePool taskManager =
+        new LogRollBackupSubprocedurePool(rss.getServerName().toString(), conf);
+    return new LogRollBackupSubprocedure(rss, member, errorDispatcher, wakeMillis, timeoutMillis,
+      taskManager);
+
+  }
+
+  /**
+   * Build the actual backup procedure runner that will do all the 'hard' work
+   */
+  public class BackupSubprocedureBuilder implements SubprocedureFactory {
+
+    @Override
+    public Subprocedure buildSubprocedure(String name, byte[] data) {
+      return LogRollRegionServerProcedureManager.this.buildSubprocedure();
+    }
+  }
+
+  @Override
+  public void initialize(RegionServerServices rss) throws IOException {
+    this.rss = rss;
+    BaseCoordinatedStateManager coordManager =
+        (BaseCoordinatedStateManager) CoordinatedStateManagerFactory.getCoordinatedStateManager(rss
+          .getConfiguration());
+    coordManager.initialize(rss);
+    this.memberRpcs =
+        coordManager
+        .getProcedureMemberRpcs(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE);
+
+    // read in the backup handler configuration properties
+    Configuration conf = rss.getConfiguration();
+    long keepAlive = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT);
+    int opThreads = conf.getInt(BACKUP_REQUEST_THREADS_KEY, BACKUP_REQUEST_THREADS_DEFAULT);
+    // create the actual cohort member
+    ThreadPoolExecutor pool =
+        ProcedureMember.defaultPool(rss.getServerName().toString(), opThreads, keepAlive);
+    this.member = new ProcedureMember(memberRpcs, pool, new BackupSubprocedureBuilder());
+  }
+
+  @Override
+  public String getProcedureSignature() {
+    return "backup-proc";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
index ae36f08..3342743 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
@@ -17,7 +17,11 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Server;
 
@@ -51,8 +55,21 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
    * Method to retrieve coordination for split log worker
    */
   public abstract  SplitLogWorkerCoordination getSplitLogWorkerCoordination();
+  
   /**
    * Method to retrieve coordination for split log manager
    */
   public abstract SplitLogManagerCoordination getSplitLogManagerCoordination();
+  /**
+   * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs}
+   */
+  public abstract ProcedureCoordinatorRpcs
+    getProcedureCoordinatorRpcs(String procType, String coordNode) throws IOException;
+  
+  /**
+   * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureMemberRpc}
+   */
+  public abstract ProcedureMemberRpcs
+    getProcedureMemberRpcs(String procType) throws IOException;
+    
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
index 3e89be7..7cf4aab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
@@ -17,9 +17,15 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
+import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /**
@@ -49,9 +55,21 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
   @Override
   public SplitLogWorkerCoordination getSplitLogWorkerCoordination() {
     return splitLogWorkerCoordination;
-    }
+  }
+
   @Override
   public SplitLogManagerCoordination getSplitLogManagerCoordination() {
     return splitLogManagerCoordination;
   }
+
+  @Override
+  public ProcedureCoordinatorRpcs getProcedureCoordinatorRpcs(String procType, String coordNode)
+      throws IOException {
+    return new ZKProcedureCoordinatorRpcs(watcher, procType, coordNode);
+  }
+
+  @Override
+  public ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws IOException {
+    return new ZKProcedureMemberRpcs(watcher, procType);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 9d9cee0..2ceeda5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -85,6 +85,9 @@ public class WALPlayer extends Configured implements Tool {
 
   private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
 
+  public WALPlayer(){
+  }
+
   protected WALPlayer(final Configuration c) {
     super(c);
   }
@@ -94,7 +97,7 @@ public class WALPlayer extends Configured implements Tool {
    * This one can be used together with {@link KeyValueSortReducer}
    */
   static class WALKeyValueMapper
-  extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
+    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
     private byte[] table;
 
     @Override
@@ -106,7 +109,9 @@ public class WALPlayer extends Configured implements Tool {
         if (Bytes.equals(table, key.getTablename().getName())) {
           for (Cell cell : value.getCells()) {
             KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-            if (WALEdit.isMetaEditFamily(kv)) continue;
+            if (WALEdit.isMetaEditFamily(kv)) {
+              continue;
+            }
             context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)), kv);
           }
         }
@@ -132,7 +137,7 @@ public class WALPlayer extends Configured implements Tool {
    * a running HBase instance.
    */
   protected static class WALMapper
-  extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
+    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
     private Map<TableName, TableName> tables = new TreeMap<TableName, TableName>();
 
     @Override
@@ -149,7 +154,9 @@ public class WALPlayer extends Configured implements Tool {
           Cell lastCell = null;
           for (Cell cell : value.getCells()) {
             // filtering WAL meta entries
-            if (WALEdit.isMetaEditFamily(cell)) continue;
+            if (WALEdit.isMetaEditFamily(cell)) {
+              continue;
+            }
 
             // Allow a subclass filter out this cell.
             if (filter(context, cell)) {
@@ -160,8 +167,12 @@ public class WALPlayer extends Configured implements Tool {
               if (lastCell == null || lastCell.getTypeByte() != cell.getTypeByte()
                   || !CellUtil.matchingRow(lastCell, cell)) {
                 // row or type changed, write out aggregate KVs.
-                if (put != null) context.write(tableOut, put);
-                if (del != null) context.write(tableOut, del);
+                if (put != null) {
+                  context.write(tableOut, put);
+                }
+                if (del != null) {
+                  context.write(tableOut, del);
+                }
                 if (CellUtil.isDelete(cell)) {
                   del = new Delete(CellUtil.cloneRow(cell));
                 } else {
@@ -177,8 +188,12 @@ public class WALPlayer extends Configured implements Tool {
             lastCell = cell;
           }
           // write residual KVs
-          if (put != null) context.write(tableOut, put);
-          if (del != null) context.write(tableOut, del);
+          if (put != null) {
+            context.write(tableOut, put);
+          }
+          if (del != null) {
+            context.write(tableOut, del);
+          }
         }
       } catch (InterruptedException e) {
         e.printStackTrace();
@@ -186,7 +201,8 @@ public class WALPlayer extends Configured implements Tool {
     }
 
     /**
-     * @param cell
+     * Filter cell
+     * @param cell cell
      * @return Return true if we are to emit this cell.
      */
     protected boolean filter(Context context, final Cell cell) {
@@ -197,9 +213,7 @@ public class WALPlayer extends Configured implements Tool {
     public void setup(Context context) throws IOException {
       String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY);
       String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY);
-      if (tablesToUse == null && tableMap == null) {
-        // Then user wants all tables.
-      } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
+      if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
         // this can only happen when WALMapper is used directly by a class other than WALPlayer
         throw new IOException("No tables or incorrect table mapping specified.");
       }
@@ -215,7 +229,9 @@ public class WALPlayer extends Configured implements Tool {
 
   void setupTime(Configuration conf, String option) throws IOException {
     String val = conf.get(option);
-    if (null == val) return;
+    if (null == val) {
+      return;
+    }
     long ms;
     try {
       // first try to parse in user friendly form
@@ -295,7 +311,8 @@ public class WALPlayer extends Configured implements Tool {
     return job;
   }
 
-  /*
+  /**
+   * Print usage
    * @param errorMsg Error message.  Can be null.
    */
   private void usage(final String errorMsg) {
@@ -305,7 +322,8 @@ public class WALPlayer extends Configured implements Tool {
     System.err.println("Usage: " + NAME + " [options] <wal inputdir> <tables> [<tableMappings>]");
     System.err.println("Read all WAL entries for <tables>.");
     System.err.println("If no tables (\"\") are specific, all tables are imported.");
-    System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported in that case.)");
+    System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported"+
+      " in that case.)");
     System.err.println("Otherwise <tables> is a comma separated list of tables.\n");
     System.err.println("The WAL entries can be mapped to new set of tables via <tableMapping>.");
     System.err.println("<tableMapping> is a command separated list of targettables.");
@@ -318,10 +336,10 @@ public class WALPlayer extends Configured implements Tool {
     System.err.println("  -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]");
     System.err.println("  -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]");
     System.err.println("   -D " + JOB_NAME_CONF_KEY
-        + "=jobName - use the specified mapreduce job name for the wal player");
+      + "=jobName - use the specified mapreduce job name for the wal player");
     System.err.println("For performance also consider the following options:\n"
-        + "  -Dmapreduce.map.speculative=false\n"
-        + "  -Dmapreduce.reduce.speculative=false");
+      + "  -Dmapreduce.map.speculative=false\n"
+      + "  -Dmapreduce.reduce.speculative=false");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/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 b4bffb4..89cfd18 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
@@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -390,6 +391,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
 
     Replication.decorateMasterConfiguration(this.conf);
+    BackupManager.decorateMasterConfiguration(this.conf);
 
     // Hack! Maps DFSClient => Master for logs.  HDFS made this
     // config param for task trackers, but we can piggyback off of it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
index 95c3ffe..b6e11ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
@@ -37,7 +37,7 @@ public abstract class RegionServerProcedureManager extends ProcedureManager {
    * @param rss Region Server service interface
    * @throws KeeperException
    */
-  public abstract void initialize(RegionServerServices rss) throws KeeperException;
+  public abstract void initialize(RegionServerServices rss) throws IOException;
 
   /**
    * Start accepting procedure requests.

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
index 0f4ea64..adb3604 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.procedure.flush.RegionServerFlushTableProcedureManager;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * Provides the globally barriered procedure framework and environment
@@ -39,7 +38,7 @@ public class RegionServerProcedureManagerHost extends
   private static final Log LOG = LogFactory
       .getLog(RegionServerProcedureManagerHost.class);
 
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     for (RegionServerProcedureManager proc : procedures) {
       LOG.debug("Procedure " + proc.getProcedureSignature() + " is initializing");
       proc.initialize(rss);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
index 085d642..3865ba9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
@@ -54,7 +54,7 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
    * @throws KeeperException if an unexpected zk error occurs
    */
   public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher,
-      String procedureClass, String coordName) throws KeeperException {
+      String procedureClass, String coordName) throws IOException {
     this.watcher = watcher;
     this.procedureType = procedureClass;
     this.coordName = coordName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
index 2e03a60..9b491fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
@@ -68,49 +68,53 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
    * @throws KeeperException if we can't reach zookeeper
    */
   public ZKProcedureMemberRpcs(final ZooKeeperWatcher watcher, final String procType)
-      throws KeeperException {
-    this.zkController = new ZKProcedureUtil(watcher, procType) {
-      @Override
-      public void nodeCreated(String path) {
-        if (!isInProcedurePath(path)) {
-          return;
-        }
+      throws IOException {
+    try {
+      this.zkController = new ZKProcedureUtil(watcher, procType) {
+        @Override
+        public void nodeCreated(String path) {
+          if (!isInProcedurePath(path)) {
+            return;
+          }
 
-        LOG.info("Received created event:" + path);
-        // if it is a simple start/end/abort then we just rewatch the node
-        if (isAcquiredNode(path)) {
-          waitForNewProcedures();
-          return;
-        } else if (isAbortNode(path)) {
-          watchForAbortedProcedures();
-          return;
+          LOG.info("Received created event:" + path);
+          // if it is a simple start/end/abort then we just rewatch the node
+          if (isAcquiredNode(path)) {
+            waitForNewProcedures();
+            return;
+          } else if (isAbortNode(path)) {
+            watchForAbortedProcedures();
+            return;
+          }
+          String parent = ZKUtil.getParent(path);
+          // if its the end barrier, the procedure can be completed
+          if (isReachedNode(parent)) {
+            receivedReachedGlobalBarrier(path);
+            return;
+          } else if (isAbortNode(parent)) {
+            abort(path);
+            return;
+          } else if (isAcquiredNode(parent)) {
+            startNewSubprocedure(path);
+          } else {
+            LOG.debug("Ignoring created notification for node:" + path);
+          }
         }
-        String parent = ZKUtil.getParent(path);
-        // if its the end barrier, the procedure can be completed
-        if (isReachedNode(parent)) {
-          receivedReachedGlobalBarrier(path);
-          return;
-        } else if (isAbortNode(parent)) {
-          abort(path);
-          return;
-        } else if (isAcquiredNode(parent)) {
-          startNewSubprocedure(path);
-        } else {
-          LOG.debug("Ignoring created notification for node:" + path);
-        }
-      }
 
-      @Override
-      public void nodeChildrenChanged(String path) {
-        if (path.equals(this.acquiredZnode)) {
-          LOG.info("Received procedure start children changed event: " + path);
-          waitForNewProcedures();
-        } else if (path.equals(this.abortZnode)) {
-          LOG.info("Received procedure abort children changed event: " + path);
-          watchForAbortedProcedures();
+        @Override
+        public void nodeChildrenChanged(String path) {
+          if (path.equals(this.acquiredZnode)) {
+            LOG.info("Received procedure start children changed event: " + path);
+            waitForNewProcedures();
+          } else if (path.equals(this.abortZnode)) {
+            LOG.info("Received procedure abort children changed event: " + path);
+            watchForAbortedProcedures();
+          }
         }
-      }
-    };
+      };
+    } catch (KeeperException e) {
+      throw new IOException(e);
+    }
   }
 
   public ZKProcedureUtil getZkController() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
index 1aa959c..bd65cc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
@@ -317,7 +317,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
    * @throws KeeperException if the zookeeper cannot be reached
    */
   @Override
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 4ab2693..0ce8ee4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -828,8 +828,8 @@ public class HRegionServer extends HasThread implements
       rspmHost = new RegionServerProcedureManagerHost();
       rspmHost.loadProcedures(conf);
       rspmHost.initialize(this);
-    } catch (KeeperException e) {
-      this.abort("Failed to reach zk cluster when creating procedure handler.", e);
+    } catch (IOException e) {
+      this.abort("Failed to reach coordination cluster when creating procedure handler.", e);
     }
     // register watcher for recovering regions
     this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
index 537329a..e56dd28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
@@ -390,7 +390,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
    * @throws KeeperException if the zookeeper cluster cannot be reached
    */
   @Override
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/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 f3f869c..31f05c2 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
@@ -96,6 +96,8 @@ import com.lmax.disruptor.TimeoutException;
 import com.lmax.disruptor.dsl.Disruptor;
 import com.lmax.disruptor.dsl.ProducerType;
 
+
+
 /**
  * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS.
  * Only one WAL is ever being written at a time.  When a WAL hits a configured maximum size,
@@ -356,7 +358,9 @@ public class FSHLog implements WAL {
     public int compare(Path o1, Path o2) {
       long t1 = getFileNumFromFileName(o1);
       long t2 = getFileNumFromFileName(o2);
-      if (t1 == t2) return 0;
+      if (t1 == t2) {
+        return 0;
+      }
       return (t1 > t2) ? 1 : -1;
     }
   };
@@ -399,7 +403,7 @@ public class FSHLog implements WAL {
    * @param root path for stored and archived wals
    * @param logDir dir where wals are stored
    * @param conf configuration to use
-   * @throws IOException
+   * @throws IOException exception
    */
   public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)
       throws IOException {
@@ -407,7 +411,7 @@ public class FSHLog implements WAL {
   }
 
   /**
-   * Create an edit log at the given <code>dir</code> location.
+   * Create an edit log at the given directory location.
    *
    * You should never have to load an existing log. If there is a log at
    * startup, it should have already been processed and deleted by the time the
@@ -422,13 +426,13 @@ public class FSHLog implements WAL {
    * be registered before we do anything else; e.g. the
    * Constructor {@link #rollWriter()}.
    * @param failIfWALExists If true IOException will be thrown if files related to this wal
-   *        already exist.
+   *     already exist.
    * @param prefix should always be hostname and port in distributed env and
-   *        it will be URL encoded before being used.
-   *        If prefix is null, "wal" will be used
+   *     it will be URL encoded before being used.
+   *     If prefix is null, "wal" will be used
    * @param suffix will be url encoded. null is treated as empty. non-empty must start with
-   *        {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
-   * @throws IOException
+   *     {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
+   * @throws IOException exception
    */
   public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
       final String archiveDir, final Configuration conf,
@@ -590,7 +594,9 @@ public class FSHLog implements WAL {
   @VisibleForTesting
   OutputStream getOutputStream() {
     FSDataOutputStream fsdos = this.hdfs_out;
-    if (fsdos == null) return null;
+    if (fsdos == null) {
+      return null;
+    }
     return fsdos.getWrappedStream();
   }
 
@@ -625,7 +631,7 @@ public class FSHLog implements WAL {
 
   /**
    * Tell listeners about pre log roll.
-   * @throws IOException
+   * @throws IOException exception
    */
   private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
   throws IOException {
@@ -638,7 +644,7 @@ public class FSHLog implements WAL {
 
   /**
    * Tell listeners about post log roll.
-   * @throws IOException
+   * @throws IOException exception
    */
   private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
   throws IOException {
@@ -651,8 +657,7 @@ public class FSHLog implements WAL {
 
   /**
    * Run a sync after opening to set up the pipeline.
-   * @param nextWriter
-   * @param startTimeNanos
+   * @param nextWriter next writer
    */
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
@@ -670,7 +675,9 @@ public class FSHLog implements WAL {
     rollWriterLock.lock();
     try {
       // Return if nothing to flush.
-      if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null;
+      if (!force && (this.writer != null && this.numEntries.get() <= 0)) {
+        return null;
+      }
       byte [][] regionsToFlush = null;
       if (this.closed) {
         LOG.debug("WAL closed. Skipping rolling of writer");
@@ -725,7 +732,7 @@ public class FSHLog implements WAL {
 
   /**
    * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
-   * @throws IOException
+   * @throws IOException exception
    */
   private void cleanOldLogs() throws IOException {
     List<Path> logsToArchive = null;
@@ -735,9 +742,13 @@ public class FSHLog implements WAL {
       Path log = e.getKey();
       Map<byte[], Long> sequenceNums = e.getValue();
       if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
-        if (logsToArchive == null) logsToArchive = new ArrayList<Path>();
+        if (logsToArchive == null) {
+          logsToArchive = new ArrayList<Path>();
+        }
         logsToArchive.add(log);
-        if (LOG.isTraceEnabled()) LOG.trace("WAL file ready for archiving " + log);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("WAL file ready for archiving " + log);
+        }
       }
     }
     if (logsToArchive != null) {
@@ -767,7 +778,9 @@ public class FSHLog implements WAL {
     if (regions != null) {
       StringBuilder sb = new StringBuilder();
       for (int i = 0; i < regions.length; i++) {
-        if (i > 0) sb.append(", ");
+        if (i > 0) {
+          sb.append(", ");
+        }
         sb.append(Bytes.toStringBinary(regions[i]));
       }
       LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs +
@@ -833,7 +846,9 @@ public class FSHLog implements WAL {
         }
       } catch (FailedSyncBeforeLogCloseException e) {
         // If unflushed/unsynced entries on close, it is reason to abort.
-        if (isUnflushedEntries()) throw e;
+        if (isUnflushedEntries()) {
+          throw e;
+        }
         LOG.warn("Failed sync-before-close but no outstanding appends; closing WAL: " +
           e.getMessage());
       }
@@ -894,7 +909,9 @@ public class FSHLog implements WAL {
             try {
               blockOnSync(syncFuture);
             } catch (IOException ioe) {
-              if (LOG.isTraceEnabled()) LOG.trace("Stale sync exception", ioe);
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("Stale sync exception", ioe);
+              }
             }
           }
         }
@@ -965,7 +982,15 @@ public class FSHLog implements WAL {
   public Path getCurrentFileName() {
     return computeFilename(this.filenum.get());
   }
-
+  
+  /**
+   * To support old API compatibility
+   * @return current file number (timestamp)
+   */
+  public long getFilenum() {
+    return filenum.get();
+  }
+  
   @Override
   public String toString() {
     return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")";

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LogUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LogUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LogUtils.java
new file mode 100644
index 0000000..26f261c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LogUtils.java
@@ -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.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public final class LogUtils {
+
+  private LogUtils() {    
+  }
+  /**
+   * Disables Zk- and HBase client logging
+   * @param log
+   */
+  public static void disableUselessLoggers(Log log) {
+    // disable zookeeper log to avoid it mess up command output
+    Logger zkLogger = Logger.getLogger("org.apache.zookeeper");
+    zkLogger.setLevel(Level.OFF);
+    // disable hbase zookeeper tool log to avoid it mess up command output
+    Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper");
+    hbaseZkLogger.setLevel(Level.OFF);
+    // disable hbase client log to avoid it mess up command output
+    Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client");
+    hbaseClientLogger.setLevel(Level.OFF);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
index 027e7a2..dd4d337 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
@@ -209,13 +209,18 @@ public class DefaultWALProvider implements WALProvider {
   @VisibleForTesting
   public static long extractFileNumFromWAL(final WAL wal) {
     final Path walName = ((FSHLog)wal).getCurrentFileName();
+    return extractFileNumFromWAL(walName);
+  }
+
+  @VisibleForTesting
+  public static long extractFileNumFromWAL(final Path walName) {
     if (walName == null) {
       throw new IllegalArgumentException("The WAL path couldn't be null");
     }
     final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
     return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
   }
-
+  
   /**
    * Pattern used to validate a WAL file name
    * see {@link #validateWALFilename(String)} for description.

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
new file mode 100644
index 0000000..84b7c78
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -0,0 +1,209 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.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.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupContext;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+/**
+ * This class is only a base for other integration-level backup tests. Do not add tests here.
+ * TestBackupSmallTests is where tests that don't require bring machines up/down should go All other
+ * tests should have their own classes and extend this one
+ */
+public class TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupBase.class);
+
+  protected static Configuration conf1;
+  protected static Configuration conf2;
+
+  protected static HBaseTestingUtility TEST_UTIL;
+  protected static HBaseTestingUtility TEST_UTIL2;
+  protected static TableName table1;
+  protected static TableName table2;
+  protected static TableName table3;
+  protected static TableName table4;
+
+  protected static TableName table1_restore = TableName.valueOf("table1_restore");
+  protected static TableName table2_restore = TableName.valueOf("table2_restore");
+  protected static TableName table3_restore = TableName.valueOf("table3_restore");
+  protected static TableName table4_restore = TableName.valueOf("table4_restore");
+
+  protected static final int NB_ROWS_IN_BATCH = 100;
+  protected static final byte[] qualName = Bytes.toBytes("q1");
+  protected static final byte[] famName = Bytes.toBytes("f");
+
+  protected static String BACKUP_ROOT_DIR = "/backupUT";
+  protected static String BACKUP_REMOTE_ROOT_DIR = "/backupUT";
+
+  protected static final String BACKUP_ZNODE = "/backup/hbase";
+  protected static final String BACKUP_SUCCEED_NODE = "complete";
+  protected static final String BACKUP_FAILED_NODE = "failed";
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL.getConfiguration().set("hbase.procedure.regionserver.classes",
+      LogRollRegionServerProcedureManager.class.getName());
+    TEST_UTIL.getConfiguration().set("hbase.procedure.master.classes",
+      LogRollMasterProcedureManager.class.getName());
+    TEST_UTIL.getConfiguration().set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    TEST_UTIL.startMiniZKCluster();
+    MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
+
+    conf1 = TEST_UTIL.getConfiguration();
+    conf2 = HBaseConfiguration.create(conf1);
+    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+    TEST_UTIL2 = new HBaseTestingUtility(conf2);
+    TEST_UTIL2.setZkCluster(miniZK);
+    TEST_UTIL.startMiniCluster();
+    TEST_UTIL2.startMiniCluster();
+    conf1 = TEST_UTIL.getConfiguration();
+
+    TEST_UTIL.startMiniMapReduceCluster();
+    BACKUP_ROOT_DIR = TEST_UTIL.getConfiguration().get("fs.defaultFS") + "/backupUT";
+    LOG.info("ROOTDIR " + BACKUP_ROOT_DIR);
+    BACKUP_REMOTE_ROOT_DIR = TEST_UTIL2.getConfiguration().get("fs.defaultFS") + "/backupUT";
+    LOG.info("REMOTE ROOTDIR " + BACKUP_REMOTE_ROOT_DIR);
+
+    createTables();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
+    TEST_UTIL2.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniMapReduceCluster();
+  }
+
+  protected static void loadTable(HTable table) throws Exception {
+
+    Put p; // 100 + 1 row to t1_syncup
+    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+      p = new Put(Bytes.toBytes("row" + i));
+      p.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      table.put(p);
+    }
+  }
+
+  protected static void createTables() throws Exception {
+
+    long tid = System.currentTimeMillis();
+    table1 = TableName.valueOf("test-" + tid);
+    HBaseAdmin ha = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor desc = new HTableDescriptor(table1);
+    HColumnDescriptor fam = new HColumnDescriptor(famName);
+    desc.addFamily(fam);
+    ha.createTable(desc);
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    HTable table = (HTable) conn.getTable(table1);
+    loadTable(table);
+    table.close();
+    table2 = TableName.valueOf("test-" + tid + 1);
+    desc = new HTableDescriptor(table2);
+    desc.addFamily(fam);
+    ha.createTable(desc);
+    table = (HTable) conn.getTable(table2);
+    loadTable(table);
+    table.close();
+    table3 = TableName.valueOf("test-" + tid + 2);
+    table = TEST_UTIL.createTable(table3, famName);
+    table.close();
+    table4 = TableName.valueOf("test-" + tid + 3);
+    table = TEST_UTIL.createTable(table4, famName);
+    table.close();
+    ha.close();
+    conn.close();
+  }
+
+  protected boolean checkSucceeded(String backupId) throws IOException {
+    BackupContext status = getBackupContext(backupId);
+    if (status == null) return false;
+    return status.getState() == BackupState.COMPLETE;
+  }
+
+  protected boolean checkFailed(String backupId) throws IOException {
+    BackupContext status = getBackupContext(backupId);
+    if (status == null) return false;
+    return status.getState() == BackupState.FAILED;
+  }
+
+  private BackupContext getBackupContext(String backupId) throws IOException {
+    Configuration conf = conf1;//BackupClientImpl.getConf();
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        BackupSystemTable table = new BackupSystemTable(connection)) {
+      BackupContext status = table.readBackupStatus(backupId);
+      return status;
+    }
+  }
+
+  protected BackupClient getBackupClient(){
+    return BackupRestoreFactory.getBackupClient(conf1);
+  }
+
+  protected RestoreClient getRestoreClient()
+  {
+    return BackupRestoreFactory.getRestoreClient(conf1);
+  }
+
+  /**
+   * Helper method
+   */
+  protected List<TableName> toList(String... args){
+    List<TableName> ret = new ArrayList<>();
+    for(int i=0; i < args.length; i++){
+      ret.add(TableName.valueOf(args[i]));
+    }
+    return ret;
+  }
+}


[47/50] [abbrv] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (v42)

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
new file mode 100644
index 0000000..7bd6e99
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
@@ -0,0 +1,702 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+
+/**
+ * A Handler to carry the operations of backup progress
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupHandler implements Callable<Void> {
+  private static final Log LOG = LogFactory.getLog(BackupHandler.class);
+
+  // backup phase
+  // for overall backup (for table list, some table may go online, while some may go offline)
+  protected static enum BackupPhase {
+    REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+  }
+
+  // backup status flag
+  public static enum BackupState {
+    WAITING, RUNNING, COMPLETE, FAILED, CANCELLED;
+  }
+
+  protected final BackupContext backupContext;
+  private final BackupManager backupManager;
+  private final Configuration conf;
+  private final Connection conn;
+
+  public BackupHandler(BackupContext backupContext,
+      BackupManager backupManager, Configuration conf, Connection connection) {
+    this.backupContext = backupContext;
+    this.backupManager = backupManager;
+    this.conf = conf;
+    this.conn = connection;
+  }
+
+  public BackupContext getBackupContext() {
+    return backupContext;
+  }
+
+  @Override
+  public Void call() throws Exception {
+    try(Admin admin = conn.getAdmin()) {
+      // overall backup begin
+      this.beginBackup(backupContext);
+      HashMap<String, Long> newTimestamps = null;
+      // handle full or incremental backup for table or table list
+      if (backupContext.getType() == BackupType.FULL) {
+        String savedStartCode = null;
+        boolean firstBackup = false;
+        // do snapshot for full table backup
+
+        try {
+          savedStartCode = backupManager.readBackupStartCode();
+          firstBackup = savedStartCode == null;
+          if (firstBackup) {
+            // This is our first backup. Let's put some marker on ZK so that we can hold the logs
+            // while we do the backup.
+            backupManager.writeBackupStartCode(0L);
+          }
+          // We roll log here before we do the snapshot. It is possible there is duplicate data
+          // in the log that is already in the snapshot. But if we do it after the snapshot, we
+          // could have data loss.
+          // A better approach is to do the roll log on each RS in the same global procedure as
+          // the snapshot.
+          LOG.info("Execute roll log procedure for full backup ...");
+          admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+            LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap<String, String>());
+          newTimestamps = backupManager.readRegionServerLastLogRollResult();
+          if (firstBackup) {
+            // Updates registered log files
+            // We record ALL old WAL files as registered, because
+            // this is a first full backup in the system and these
+            // files are not needed for next incremental backup
+            List<String> logFiles = BackupUtil.getWALFilesOlderThan(conf, newTimestamps);
+            backupManager.recordWALFiles(logFiles);
+          }
+          this.snapshotForFullBackup(backupContext);
+        } catch (BackupException e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected BackupException : ");
+          return null;
+        }
+
+        // update the faked progress currently for snapshot done
+        updateProgress(backupContext, backupManager, 10, 0);
+        // do snapshot copy
+        try {
+          this.snapshotCopy(backupContext);
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected BackupException : ");
+          return null;
+        }
+        // Updates incremental backup table set
+        backupManager.addIncrementalBackupTableSet(backupContext.getTables());
+
+      } else if (backupContext.getType() == BackupType.INCREMENTAL) {
+        LOG.debug("For incremental backup, current table set is "
+            + backupManager.getIncrementalBackupTableSet());
+        // do incremental table backup preparation
+        backupContext.setPhase(BackupPhase.PREPARE_INCREMENTAL);
+        // avoid action if has been cancelled
+        if (backupContext.isCancelled()) {
+          return null;
+        }
+        try {
+          IncrementalBackupManager incrBackupManager = new IncrementalBackupManager(backupManager);
+
+          newTimestamps = incrBackupManager.getIncrBackupLogFileList(backupContext);
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected Exception : ");
+          return null;
+        }
+        // update the faked progress currently for incremental preparation done
+        updateProgress(backupContext, backupManager, 10, 0);
+
+        // do incremental copy
+        try {
+          // copy out the table and region info files for each table
+          BackupUtil.copyTableRegionInfo(backupContext, conf);
+          this.incrementalCopy(backupContext);
+          // Save list of WAL files copied
+          backupManager.recordWALFiles(backupContext.getIncrBackupFileList());
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected exception doing incremental copy : ");
+          return null;
+        }
+      }
+
+      // set overall backup status: complete. Here we make sure to complete the backup. After this
+      // checkpoint, even if entering cancel process, will let the backup finished
+      backupContext.setState(BackupState.COMPLETE);
+
+      if (backupContext.getType() == BackupType.INCREMENTAL) {
+        // Set the previousTimestampMap which is before this current log roll to the manifest.
+        HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
+            backupManager.readLogTimestampMap();
+        backupContext.setIncrTimestampMap(previousTimestampMap);
+      }
+
+      // The table list in backupContext is good for both full backup and incremental backup.
+      // For incremental backup, it contains the incremental backup table set.
+      backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
+
+      HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+          backupManager.readLogTimestampMap();
+
+      Long newStartCode =
+          BackupUtility.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap));
+      backupManager.writeBackupStartCode(newStartCode);
+
+      // backup complete
+      this.completeBackup(backupContext);
+    } catch (Exception e) {
+      // even during completing backup (#completeBackup(backupContext)), exception may occur, or
+      // exception occur during other process, fail the backup finally
+      this.failBackup(backupContext, e, "Error caught during backup progress: ");
+    }
+    return null;
+  }
+
+  /**
+   * Begin the overall backup.
+   * @param backupContext backup context
+   * @throws IOException exception
+   */
+  private void beginBackup(BackupContext backupContext) throws IOException {
+    // set the start timestamp of the overall backup
+    long startTs = EnvironmentEdgeManager.currentTime();
+    backupContext.setStartTs(startTs);
+    // set overall backup status: ongoing
+    backupContext.setState(BackupState.RUNNING);
+    LOG.info("Backup " + backupContext.getBackupId() + " started at " + startTs + ".");
+
+    backupManager.updateBackupStatus(backupContext);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup session " + backupContext.getBackupId() + " has been started.");
+    }
+  }
+
+  /**
+   * Snapshot for full table backup.
+   * @param backupContext backup context
+   * @throws IOException exception
+   */
+  private void snapshotForFullBackup(BackupContext backupContext) throws IOException {
+    LOG.info("HBase snapshot full backup for " + backupContext.getBackupId());
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    try (Admin admin = conn.getAdmin()) {
+      // we do HBase snapshot for tables in the table list one by one currently
+      for (TableName table : backupContext.getTables()) {
+        // avoid action if it has been cancelled
+        if (backupContext.isCancelled()) {
+          return;
+        }
+
+        HBaseProtos.SnapshotDescription backupSnapshot;
+
+        // wrap a SnapshotDescription for offline/online snapshot
+        backupSnapshot = this.wrapSnapshotDescription(table);
+
+        try {
+          // Kick off snapshot for backup
+          admin.snapshot(backupSnapshot);
+        } catch (Exception e) {
+          LOG.error("Snapshot failed to create " + getMessage(e));
+
+          // currently, we fail the overall backup if any table in the list failed, so throw the
+          // exception out for overall backup failing
+          throw new BackupException("Backup snapshot failed on table " + table, e);
+        }
+
+        // set the snapshot name in BackupStatus of this table, only after snapshot success.
+        backupContext.setSnapshotName(table, backupSnapshot.getName());
+      }
+    }
+  }
+
+  /**
+   * Fail the overall backup.
+   * @param backupContext backup context
+   * @param e exception
+   * @throws Exception exception
+   */
+  private void failBackup(BackupContext backupContext, Exception e, String msg) throws Exception {
+    LOG.error(msg + getMessage(e));
+    // If this is a cancel exception, then we've already cleaned.
+
+    if (this.backupContext.getState().equals(BackupState.CANCELLED)) {
+      return;
+    }
+
+    // set the failure timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+
+    // set failure message
+    backupContext.setFailedMsg(e.getMessage());
+
+    // set overall backup status: failed
+    backupContext.setState(BackupState.FAILED);
+
+    // compose the backup failed data
+    String backupFailedData =
+        "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs()
+        + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase()
+        + ",failedmessage=" + backupContext.getFailedMsg();
+    LOG.error(backupFailedData);
+
+    backupManager.updateBackupStatus(backupContext);
+
+    // if full backup, then delete HBase snapshots if there already have snapshots taken
+    // and also clean up export snapshot log files if exist
+    if (backupContext.getType() == BackupType.FULL) {
+      this.deleteSnapshot(backupContext);
+      this.cleanupExportSnapshotLog();
+    } /*
+     * else { // support incremental backup code in future jira // TODO. See HBASE-14124 }
+     */
+
+    // clean up the uncompleted data at target directory if the ongoing backup has already entered
+    // the copy phase
+    // For incremental backup, DistCp logs will be cleaned with the targetDir.
+    this.cleanupTargetDir();
+
+    LOG.info("Backup " + backupContext.getBackupId() + " failed.");
+  }
+
+  /**
+   * Update the ongoing back token znode with new progress.
+   * @param backupContext backup context
+   * 
+   * @param newProgress progress
+   * @param bytesCopied bytes copied
+   * @throws NoNodeException exception
+   */
+  public static void updateProgress(BackupContext backupContext, BackupManager backupManager,
+      int newProgress, long bytesCopied) throws IOException {
+    // compose the new backup progress data, using fake number for now
+    String backupProgressData = newProgress + "%";
+
+    backupContext.setProgress(newProgress);
+    backupManager.updateBackupStatus(backupContext);
+    LOG.debug("Backup progress data \"" + backupProgressData
+      + "\" has been updated to hbase:backup for " + backupContext.getBackupId());
+  }
+
+  /**
+   * Complete the overall backup.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  private void completeBackup(BackupContext backupContext) throws Exception {
+
+    // set the complete timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+    // set overall backup status: complete
+    backupContext.setState(BackupState.COMPLETE);
+    // add and store the manifest for the backup
+    this.addManifest(backupContext);
+
+    // after major steps done and manifest persisted, do convert if needed for incremental backup
+    /* in-fly convert code here, provided by future jira */
+    LOG.debug("in-fly convert code here, provided by future jira");
+
+    // compose the backup complete data
+    String backupCompleteData =
+        this.obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs()
+        + ",completets=" + backupContext.getEndTs() + ",bytescopied="
+        + backupContext.getTotalBytesCopied();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData);
+    }
+    backupManager.updateBackupStatus(backupContext);
+
+    // when full backup is done:
+    // - delete HBase snapshot
+    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
+    // snapshots
+    if (backupContext.getType() == BackupType.FULL) {
+      this.deleteSnapshot(backupContext);
+      this.cleanupExportSnapshotLog();
+    } else if (backupContext.getType() == BackupType.INCREMENTAL) {
+      this.cleanupDistCpLog();
+    }
+
+    LOG.info("Backup " + backupContext.getBackupId() + " completed.");
+  }
+
+  /**
+   * Get backup request meta data dir as string.
+   * @param backupContext backup context
+   * @return meta data dir
+   */
+  private String obtainBackupMetaDataStr(BackupContext backupContext) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("type=" + backupContext.getType() + ",tablelist=");
+    for (TableName table : backupContext.getTables()) {
+      sb.append(table + ";");
+    }
+    if (sb.lastIndexOf(";") > 0) {
+      sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
+    }
+    sb.append(",targetRootDir=" + backupContext.getTargetRootDir());
+
+    return sb.toString();
+  }
+
+  /**
+   * Do snapshot copy.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  private void snapshotCopy(BackupContext backupContext) throws Exception {
+    LOG.info("Snapshot copy is starting.");
+
+    // set overall backup phase: snapshot_copy
+    backupContext.setPhase(BackupPhase.SNAPSHOTCOPY);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    // call ExportSnapshot to copy files based on hbase snapshot for backup
+    // ExportSnapshot only support single snapshot export, need loop for multiple tables case
+    BackupCopyService copyService = BackupRestoreFactory.getBackupCopyService(conf);
+
+    // number of snapshots matches number of tables
+    float numOfSnapshots = backupContext.getSnapshotNames().size();
+
+    LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
+
+    for (TableName table : backupContext.getTables()) {
+      // Currently we simply set the sub copy tasks by counting the table snapshot number, we can
+      // calculate the real files' size for the percentage in the future.
+      // TODO this below
+      // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots);
+      int res = 0;
+      String[] args = new String[4];
+      args[0] = "-snapshot";
+      args[1] = backupContext.getSnapshotName(table);
+      args[2] = "-copy-to";
+      args[3] = backupContext.getBackupStatus(table).getTargetDir();
+
+      LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
+      res = copyService.copy(backupContext, backupManager, conf, BackupCopyService.Type.FULL, args);
+      // if one snapshot export failed, do not continue for remained snapshots
+      if (res != 0) {
+        LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + ".");
+
+        throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3]
+            + " with reason code " + res);
+      }
+
+      LOG.info("Snapshot copy " + args[1] + " finished.");
+    }
+  }
+
+  /**
+   * Wrap a SnapshotDescription for a target table.
+   * @param table table
+   * @return a SnapshotDescription especially for backup.
+   */
+  private SnapshotDescription wrapSnapshotDescription(TableName tableName) {
+    // Mock a SnapshotDescription from backupContext to call SnapshotManager function,
+    // Name it in the format "snapshot_<timestamp>_<table>"
+    HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
+    builder.setTable(tableName.getNameAsString());
+    builder.setName("snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
+        + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString());
+    HBaseProtos.SnapshotDescription backupSnapshot = builder.build();
+
+    LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName()
+      + " from backupContext to request snapshot for backup.");
+
+    return backupSnapshot;
+  }
+
+  /**
+   * Delete HBase snapshot for backup.
+   * @param backupCtx backup context
+   * @throws Exception exception
+   */
+  private void deleteSnapshot(BackupContext backupCtx) throws IOException {
+
+    LOG.debug("Trying to delete snapshot for full backup.");
+    Connection conn = null;
+    Admin admin = null;
+    try {
+      conn = ConnectionFactory.createConnection(conf);
+      admin = conn.getAdmin();
+      for (String snapshotName : backupCtx.getSnapshotNames()) {
+        if (snapshotName == null) {
+          continue;
+        }
+        LOG.debug("Trying to delete snapshot: " + snapshotName);
+        admin.deleteSnapshot(snapshotName);
+        LOG.debug("Deleting the snapshot " + snapshotName + " for backup "
+            + backupCtx.getBackupId() + " succeeded.");
+      }
+    } finally {
+      if (admin != null) {
+        admin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "exportSnapshot-", which are generated when exporting
+   * snapshots.
+   * @throws IOException exception
+   */
+  private void cleanupExportSnapshotLog() throws IOException {
+    FileSystem fs = FSUtils.getCurrentFileSystem(conf);
+    Path stagingDir =
+        new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
+          .toString()));
+    FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("exportSnapshot-")) {
+        LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
+        if (FSUtils.delete(fs, file.getPath(), true) == false) {
+          LOG.warn("Can not delete " + file.getPath());
+        }
+      }
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying
+   * hlogs.
+   * @throws IOException exception
+   */
+  private void cleanupDistCpLog() throws IOException {
+    Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent();
+    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
+    FileStatus[] files = FSUtils.listStatus(fs, rootPath);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("_distcp_logs")) {
+        LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
+        FSUtils.delete(fs, file.getPath(), true);
+      }
+    }
+  }
+
+  /**
+   * Clean up the uncompleted data at target directory if the ongoing backup has already entered the
+   * copy phase.
+   */
+  private void cleanupTargetDir() {
+    try {
+      // clean up the uncompleted data at target directory if the ongoing backup has already entered
+      // the copy phase
+      LOG.debug("Trying to cleanup up target dir. Current backup phase: "
+          + backupContext.getPhase());
+      if (backupContext.getPhase().equals(BackupPhase.SNAPSHOTCOPY)
+          || backupContext.getPhase().equals(BackupPhase.INCREMENTAL_COPY)
+          || backupContext.getPhase().equals(BackupPhase.STORE_MANIFEST)) {
+        FileSystem outputFs =
+            FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf);
+
+        // now treat one backup as a transaction, clean up data that has been partially copied at
+        // table level
+        for (TableName table : backupContext.getTables()) {
+          Path targetDirPath =
+              new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(),
+                backupContext.getBackupId(), table));
+          if (outputFs.delete(targetDirPath, true)) {
+            LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString()
+              + " done.");
+          } else {
+            LOG.info("No data has been copied to " + targetDirPath.toString() + ".");
+          }
+
+          Path tableDir = targetDirPath.getParent();
+          FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
+          if (backups == null || backups.length == 0) {
+            outputFs.delete(tableDir, true);
+            LOG.debug(tableDir.toString() + " is empty, remove it.");
+          }
+        }
+      }
+
+    } catch (IOException e1) {
+      LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at "
+          + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
+    }
+  }
+
+  /**
+   * Add manifest for the current backup. The manifest is stored
+   * within the table backup directory.
+   * @param backupContext The current backup context
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  private void addManifest(BackupContext backupContext) throws IOException, BackupException {
+    // set the overall backup phase : store manifest
+    backupContext.setPhase(BackupPhase.STORE_MANIFEST);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    BackupManifest manifest;
+
+    // Since we have each table's backup in its own directory structure,
+    // we'll store its manifest with the table directory.
+    for (TableName table : backupContext.getTables()) {
+      manifest = new BackupManifest(backupContext, table);
+      ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext, table);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+
+      if (backupContext.getType() == BackupType.INCREMENTAL) {
+        // We'll store the log timestamps for this table only in its manifest.
+        HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
+            new HashMap<TableName, HashMap<String, Long>>();
+        tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table));
+        manifest.setIncrTimestampMap(tableTimestampMap);
+      }
+      manifest.store(conf);
+    }
+
+    // For incremental backup, we store a overall manifest in
+    // <backup-root-dir>/WALs/<backup-id>
+    // This is used when created the next incremental backup
+    if (backupContext.getType() == BackupType.INCREMENTAL) {
+      manifest = new BackupManifest(backupContext);
+      // set the table region server start and end timestamps for incremental backup
+      manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap());
+      ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+      manifest.store(conf);
+    }
+  }
+
+  /**
+   * Do incremental copy.
+   * @param backupContext backup context
+   */
+  private void incrementalCopy(BackupContext backupContext) throws Exception {
+
+    LOG.info("Incremental copy is starting.");
+
+    // set overall backup phase: incremental_copy
+    backupContext.setPhase(BackupPhase.INCREMENTAL_COPY);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    // get incremental backup file list and prepare parms for DistCp
+    List<String> incrBackupFileList = backupContext.getIncrBackupFileList();
+    // filter missing files out (they have been copied by previous backups)
+    incrBackupFileList = filterMissingFiles(incrBackupFileList);
+    String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+    strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
+
+    BackupCopyService copyService = BackupRestoreFactory.getBackupCopyService(conf);
+    int res = copyService.copy(backupContext, backupManager, conf,
+      BackupCopyService.Type.INCREMENTAL, strArr);
+
+    if (res != 0) {
+      LOG.error("Copy incremental log files failed with return code: " + res + ".");
+      throw new IOException("Failed of Hadoop Distributed Copy from " + incrBackupFileList + " to "
+          + backupContext.getHLogTargetDir());
+    }
+    LOG.info("Incremental copy from " + incrBackupFileList + " to "
+        + backupContext.getHLogTargetDir() + " finished.");
+
+  }
+
+  private List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for(String file : incrBackupFileList){
+      if(fs.exists(new Path(file))){
+        list.add(file);
+      } else{
+        LOG.warn("Can't find file: "+file);
+      }
+    }
+    return list;
+  }
+
+  private String getMessage(Exception e) {
+    String msg = e.getMessage();
+    if (msg == null || msg.equals("")) {
+      msg = e.getClass().getName();
+    }
+    return msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
new file mode 100644
index 0000000..a4b0a0a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -0,0 +1,512 @@
+/**
+ * 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.backup.impl;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+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.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupHandler.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+
+/**
+ * Handles backup requests on server-side, creates backup context records in hbase:backup
+ * to keep track backup. The timestamps kept in hbase:backup table will be used for future
+ * incremental backup. Creates BackupContext and DispatchRequest.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupManager implements Closeable {
+  private static final Log LOG = LogFactory.getLog(BackupManager.class);
+
+  private Configuration conf = null;
+  private BackupContext backupContext = null;
+
+  private ExecutorService pool = null;
+
+  private boolean backupComplete = false;
+
+  private BackupSystemTable systemTable;
+
+  private final Connection conn;
+
+  /**
+   * Backup manager constructor.
+   * @param conf configuration
+   * @throws IOException exception
+   */
+  public BackupManager(Configuration conf) throws IOException {
+    if (!conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
+      throw new BackupException("HBase backup is not enabled. Check your " +
+          HConstants.BACKUP_ENABLE_KEY + " setting.");
+    }
+    this.conf = conf;
+    this.conn = ConnectionFactory.createConnection(conf); // TODO: get Connection from elsewhere?
+    this.systemTable = new BackupSystemTable(conn);
+    Runtime.getRuntime().addShutdownHook(new ExitHandler());
+  }
+
+  /**
+   * This method modifies the master's configuration in order to inject backup-related features
+   * @param conf configuration
+   */
+  public static void decorateMasterConfiguration(Configuration conf) {
+    if (!isBackupEnabled(conf)) {
+      return;
+    }
+    String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
+    String cleanerClass = BackupLogCleaner.class.getCanonicalName();
+    if (!plugins.contains(cleanerClass)) {
+      conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Added log cleaner: " + cleanerClass);
+    }
+  }
+
+  private static boolean isBackupEnabled(Configuration conf) {
+    return conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT);
+  }
+
+  // TODO: remove this on the server side
+  private class ExitHandler extends Thread {
+    public ExitHandler() {
+      super("Backup Manager Exit Handler");
+    }
+
+    @Override
+    public void run() {
+      if (backupContext != null && !backupComplete) {
+
+        // program exit and backup is not complete, then mark as cancelled to avoid submitted backup
+        // handler's taking further action
+        backupContext.setCancelled(true);
+
+        LOG.debug("Backup is cancelled due to force program exiting.");
+        try {
+          cancelBackup(backupContext.getBackupId());
+        } catch (Exception e) {
+          String msg = e.getMessage();
+          if (msg == null || msg.equals("")) {
+            msg = e.getClass().getName();
+          }
+          LOG.error("Failed to cancel backup " + backupContext.getBackupId() + " due to " + msg);
+        }
+      }
+      close();
+    }
+  }
+
+  /**
+   * Get configuration
+   * @return configuration
+   */
+  Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Cancel the ongoing backup via backup id.
+   * @param backupId The id of the ongoing backup to be cancelled
+   * @throws Exception exception
+   */
+  private void cancelBackup(String backupId) throws Exception {
+    // TODO: will be implemented in Phase 2: HBASE-14125
+    LOG.debug("Try to cancel the backup " + backupId + ". the feature is NOT implemented yet");
+
+  }
+
+  /**
+   * Stop all the work of backup.
+   */
+  @Override
+  public void close() {
+    // currently, we shutdown now for all ongoing back handlers, we may need to do something like
+    // record the failed list somewhere later
+    if (this.pool != null) {
+      this.pool.shutdownNow();
+    }
+    if (systemTable != null) {
+      try{
+        systemTable.close();
+      } catch(Exception e){
+        LOG.error(e);
+      }
+    }
+    if (conn != null) {
+      try {
+        conn.close();
+      } catch (IOException e) {
+        LOG.error(e);
+      }
+    }
+  }
+
+  /**
+   * Create a BackupContext based on input backup request.
+   * @param backupId backup id
+   * @param type    type
+   * @param tablelist table list
+   * @param targetRootDir root dir
+   * @param snapshot snapshot name
+   * @return BackupContext context
+   * @throws BackupException exception
+   */
+  protected BackupContext createBackupContext(String backupId, BackupType type,
+      List<TableName> tableList, String targetRootDir) throws BackupException {
+
+    if (targetRootDir == null) {
+      throw new BackupException("Wrong backup request parameter: target backup root directory");
+    }
+
+    if (type == BackupType.FULL && tableList == null) {
+      // If table list is null for full backup, which means backup all tables. Then fill the table
+      // list with all user tables from meta. It no table available, throw the request exception.
+
+      HTableDescriptor[] htds = null;
+      try (Admin hbadmin = conn.getAdmin()) {
+        htds = hbadmin.listTables();
+      } catch (Exception e) {
+        throw new BackupException(e);
+      }
+
+      if (htds == null) {
+        throw new BackupException("No table exists for full backup of all tables.");
+      } else {
+        tableList = new ArrayList<>();
+        for (HTableDescriptor hTableDescriptor : htds) {
+          tableList.add(hTableDescriptor.getTableName());
+        }
+
+        LOG.info("Full backup all the tables available in the cluster: " + tableList);
+      }
+    }
+
+    // there are one or more tables in the table list
+    return new BackupContext(backupId, type, tableList.toArray(new TableName[tableList.size()]),
+      targetRootDir);
+  }
+
+  /**
+   * Check if any ongoing backup. Currently, we only reply on checking status in hbase:backup. We
+   * need to consider to handle the case of orphan records in the future. Otherwise, all the coming
+   * request will fail.
+   * @return the ongoing backup id if on going backup exists, otherwise null
+   * @throws IOException exception
+   */
+  private String getOngoingBackupId() throws IOException {
+
+    ArrayList<BackupContext> sessions = systemTable.getBackupContexts(BackupState.RUNNING);
+    if (sessions.size() == 0) {
+      return null;
+    }
+    return sessions.get(0).getBackupId();
+  }
+
+  /**
+   * Start the backup manager service.
+   * @throws IOException exception
+   */
+  public void initialize() throws IOException {
+    String ongoingBackupId = this.getOngoingBackupId();
+    if (ongoingBackupId != null) {
+      LOG.info("There is a ongoing backup " + ongoingBackupId
+          + ". Can not launch new backup until no ongoing backup remains.");
+      throw new BackupException("There is ongoing backup.");
+    }
+
+    // Initialize thread pools
+    int nrThreads = this.conf.getInt("hbase.backup.threads.max", 1);
+    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
+    builder.setNameFormat("BackupHandler-%1$d");
+    this.pool =
+        new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
+            new LinkedBlockingQueue<Runnable>(), builder.build());
+    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
+  }
+
+  /**
+   * Dispatch and handle a backup request.
+   * @param backupContext backup context
+   * @throws BackupException exception
+   */
+  public void dispatchRequest(BackupContext backupContext) throws BackupException {
+
+    this.backupContext = backupContext;
+
+    LOG.info("Got a backup request: " + "Type: " + backupContext.getType() + "; Tables: "
+        + backupContext.getTableNames() + "; TargetRootDir: " + backupContext.getTargetRootDir());
+
+    // dispatch the request to a backup handler and put it handler map
+
+    BackupHandler handler = new BackupHandler(this.backupContext, this, conf, this.conn);
+    Future<Void> future = this.pool.submit(handler);
+    // wait for the execution to complete
+    try {
+      future.get();
+    } catch (InterruptedException e) {
+      throw new BackupException(e);
+    } catch (CancellationException e) {
+      throw new BackupException(e);
+    } catch (ExecutionException e) {
+      throw new BackupException(e);
+    }
+
+    // mark the backup complete for exit handler's processing
+    backupComplete = true;
+
+    LOG.info("Backup request " + backupContext.getBackupId() + " has been executed.");
+  }
+
+  /**
+   * Get direct ancestors of the current backup.
+   * @param backupCtx The backup context for the current backup
+   * @return The ancestors for the current backup
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  protected ArrayList<BackupImage> getAncestors(BackupContext backupCtx) throws IOException,
+      BackupException {
+    LOG.debug("Getting the direct ancestors of the current backup ...");
+
+    ArrayList<BackupImage> ancestors = new ArrayList<BackupImage>();
+
+    // full backup does not have ancestor
+    if (backupCtx.getType() == BackupType.FULL) {
+      LOG.debug("Current backup is a full backup, no direct ancestor for it.");
+      return ancestors;
+    }
+
+    // get all backup history list in descending order
+
+    ArrayList<BackupCompleteData> allHistoryList = getBackupHistory();
+    for (BackupCompleteData backup : allHistoryList) {
+      BackupImage image =
+          new BackupImage(backup.getBackupToken(), BackupType.valueOf(backup.getType()),
+            backup.getBackupRootPath(),
+              backup.getTableList(), Long.parseLong(backup.getStartTime()), Long.parseLong(backup
+                  .getEndTime()));
+      // add the full backup image as an ancestor until the last incremental backup
+      if (backup.getType().equals(BackupType.FULL.toString())) {
+        // check the backup image coverage, if previous image could be covered by the newer ones,
+        // then no need to add
+        if (!BackupManifest.canCoverImage(ancestors, image)) {
+          ancestors.add(image);
+        }
+      } else {
+        // found last incremental backup, if previously added full backup ancestor images can cover
+        // it, then this incremental ancestor is not the dependent of the current incremental
+        // backup, that is to say, this is the backup scope boundary of current table set.
+        // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing
+        // incremental backup
+        if (BackupManifest.canCoverImage(ancestors, image)) {
+          LOG.debug("Met the backup boundary of the current table set. "
+              + "The root full backup images for the current backup scope:");
+          for (BackupImage image1 : ancestors) {
+            LOG.debug("  BackupId: " + image1.getBackupId() + ", Backup directory: "
+                + image1.getRootDir());
+          }
+        } else {
+          Path logBackupPath =
+              HBackupFileSystem.getLogBackupPath(backup.getBackupRootPath(),
+                backup.getBackupToken());
+          LOG.debug("Current backup has an incremental backup ancestor, "
+              + "touching its image manifest in " + logBackupPath.toString()
+              + " to construct the dependency.");
+
+          BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath);
+          BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage();
+          ancestors.add(lastIncrImage);
+
+          LOG.debug("Last dependent incremental backup image information:");
+          LOG.debug("  Token: " + lastIncrImage.getBackupId());
+          LOG.debug("  Backup directory: " + lastIncrImage.getRootDir());
+        }
+      }
+    }
+    LOG.debug("Got " + ancestors.size() + " ancestors for the current backup.");
+    return ancestors;
+  }
+
+  /**
+   * Get the direct ancestors of this backup for one table involved.
+   * @param backupContext backup context
+   * @param table table
+   * @return backupImages on the dependency list
+   * @throws BackupException exception
+   * @throws IOException exception
+   */
+  protected ArrayList<BackupImage> getAncestors(BackupContext backupContext, TableName table)
+      throws BackupException, IOException {
+    ArrayList<BackupImage> ancestors = getAncestors(backupContext);
+    ArrayList<BackupImage> tableAncestors = new ArrayList<BackupImage>();
+    for (BackupImage image : ancestors) {
+      if (image.hasTable(table)) {
+        tableAncestors.add(image);
+        if (image.getType() == BackupType.FULL) {
+          break;
+        }
+      }
+    }
+    return tableAncestors;
+  }
+
+  /*
+   * hbase:backup operations
+   */
+
+  /**
+   * Updates status (state) of a backup session in a persistent store
+   * @param context context
+   * @throws IOException exception
+   */
+  public void updateBackupStatus(BackupContext context) throws IOException {
+    systemTable.updateBackupStatus(context);
+  }
+
+  /**
+   * Read the last backup start code (timestamp) of last successful backup. Will return null
+   * if there is no startcode stored in hbase:backup or the value is of length 0. These two
+   * cases indicate there is no successful backup completed so far.
+   * @return the timestamp of a last successful backup
+   * @throws IOException exception
+   */
+  public String readBackupStartCode() throws IOException {
+    return systemTable.readBackupStartCode();
+  }
+
+  /**
+   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(Long startCode) throws IOException {
+    systemTable.writeBackupStartCode(startCode);
+  }
+
+  /**
+   * Get the RS log information after the last log roll from hbase:backup.
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> readRegionServerLastLogRollResult() throws IOException {
+    return systemTable.readRegionServerLastLogRollResult();
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public ArrayList<BackupCompleteData> getBackupHistory() throws IOException {
+    return systemTable.getBackupHistory();
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
+   * incremental backup. Each table may have a different set of log timestamps. The saved timestamp
+   * is of the last log file that was backed up already.
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void writeRegionServerLogTimestamp(Set<TableName> tables,
+      HashMap<String, Long> newTimestamps) throws IOException {
+    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps);
+  }
+
+  /**
+   * Read the timestamp for each region server log after the last successful backup. Each table has
+   * its own set of the timestamps.
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<TableName, HashMap<String, Long>> readLogTimestampMap() throws IOException {
+    return systemTable.readLogTimestampMap();
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<TableName> getIncrementalBackupTableSet() throws IOException {
+    return systemTable.getIncrementalBackupTableSet();
+  }
+
+  /**
+   * Adds set of tables to overall incremental backup table set
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<TableName> tables) throws IOException {
+    systemTable.addIncrementalBackupTableSet(tables);
+  }
+
+  /**
+   * Saves list of WAL files after incremental backup operation. These files will be stored until
+   * TTL expiration and are used by Backup Log Cleaner plugin to determine which WAL files can be
+   * safely purged.
+   */
+  public void recordWALFiles(List<String> files) throws IOException {
+    systemTable.addWALFiles(files, backupContext.getBackupId());
+  }
+
+  /**
+   * Get WAL files iterator
+   * @return WAL files iterator from hbase:backup
+   * @throws IOException
+   */
+  public Iterator<String> getWALFilesFromBackupSystem() throws IOException {
+    return  systemTable.getWALFilesIterator();
+  }
+
+  public Connection getConnection() {
+    return conn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
new file mode 100644
index 0000000..6264fc5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -0,0 +1,762 @@
+/**
+ * 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.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+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.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+
+/**
+ * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
+ * as manifest file together with data. So that each backup image will contain all the info needed
+ * for restore.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupManifest {
+
+  private static final Log LOG = LogFactory.getLog(BackupManifest.class);
+
+  // manifest file name
+  public static final String MANIFEST_FILE_NAME = ".backup.manifest";
+
+  // manifest file version, current is 1.0
+  public static final String MANIFEST_VERSION = "1.0";
+
+  // backup image, the dependency graph is made up by series of backup images
+
+  public static class BackupImage implements Comparable<BackupImage> {
+
+    private String backupId;
+    private BackupType type;
+    private String rootDir;
+    private List<TableName> tableList;
+    private long startTs;
+    private long completeTs;
+    private ArrayList<BackupImage> ancestors;
+
+    public BackupImage() {
+      super();
+    }
+
+    public BackupImage(String backupId, BackupType type, String rootDir,
+        List<TableName> tableList, long startTs, long completeTs) {
+      this.backupId = backupId;
+      this.type = type;
+      this.rootDir = rootDir;
+      this.tableList = tableList;
+      this.startTs = startTs;
+      this.completeTs = completeTs;
+    }
+
+    static BackupImage fromProto(BackupProtos.BackupImage im) {
+      String backupId = im.getBackupId();
+      String rootDir = im.getRootDir();
+      long startTs = im.getStartTs();
+      long completeTs = im.getCompleteTs();
+      List<HBaseProtos.TableName> tableListList = im.getTableListList();
+      List<TableName> tableList = new ArrayList<TableName>();
+      for(HBaseProtos.TableName tn : tableListList) {
+        tableList.add(ProtobufUtil.toTableName(tn));
+      }
+      BackupType type =
+          im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL:
+            BackupType.INCREMENTAL;
+
+      return new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
+    }
+
+    BackupProtos.BackupImage toProto() {
+      BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder();
+      builder.setBackupId(backupId);
+      builder.setCompleteTs(completeTs);
+      builder.setStartTs(startTs);
+      builder.setRootDir(rootDir);
+      if (type == BackupType.FULL) {
+        builder.setBackupType(BackupProtos.BackupType.FULL);
+      } else{
+        builder.setBackupType(BackupProtos.BackupType.INCREMENTAL);
+      }
+
+      for (TableName name: tableList) {
+        builder.addTableList(ProtobufUtil.toProtoTableName(name));
+      }
+
+      if (ancestors != null){
+        for (BackupImage im: ancestors){
+          builder.addAncestors(im.toProto());
+        }
+      }
+
+      return builder.build();
+    }
+
+    public String getBackupId() {
+      return backupId;
+    }
+
+    public void setBackupId(String backupId) {
+      this.backupId = backupId;
+    }
+
+    public BackupType getType() {
+      return type;
+    }
+
+    public void setType(BackupType type) {
+      this.type = type;
+    }
+
+    public String getRootDir() {
+      return rootDir;
+    }
+
+    public void setRootDir(String rootDir) {
+      this.rootDir = rootDir;
+    }
+
+    public List<TableName> getTableNames() {
+      return tableList;
+    }
+
+    public void setTableList(List<TableName> tableList) {
+      this.tableList = tableList;
+    }
+
+    public long getStartTs() {
+      return startTs;
+    }
+
+    public void setStartTs(long startTs) {
+      this.startTs = startTs;
+    }
+
+    public long getCompleteTs() {
+      return completeTs;
+    }
+
+    public void setCompleteTs(long completeTs) {
+      this.completeTs = completeTs;
+    }
+
+    public ArrayList<BackupImage> getAncestors() {
+      if (this.ancestors == null) {
+        this.ancestors = new ArrayList<BackupImage>();
+      }
+      return this.ancestors;
+    }
+
+    public void addAncestor(BackupImage backupImage) {
+      this.getAncestors().add(backupImage);
+    }
+
+    public boolean hasAncestor(String token) {
+      for (BackupImage image : this.getAncestors()) {
+        if (image.getBackupId().equals(token)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public boolean hasTable(TableName table) {
+      for (TableName t : tableList) {
+        if (t.getNameAsString().equals(table)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public int compareTo(BackupImage other) {
+      String thisBackupId = this.getBackupId();
+      String otherBackupId = other.getBackupId();
+      Long thisTS = new Long(thisBackupId.substring(thisBackupId.lastIndexOf("_") + 1));
+      Long otherTS = new Long(otherBackupId.substring(otherBackupId.lastIndexOf("_") + 1));
+      return thisTS.compareTo(otherTS);
+    }
+  }
+
+  // manifest version
+  private String version = MANIFEST_VERSION;
+
+  // hadoop hbase configuration
+  protected Configuration config = null;
+
+  // backup root directory
+  private String rootDir = null;
+
+  // backup image directory
+  private String tableBackupDir = null;
+
+  // backup log directory if this is an incremental backup
+  private String logBackupDir = null;
+
+  // backup token
+  private String backupId;
+
+  // backup type, full or incremental
+  private BackupType type;
+
+  // the table list for the backup
+  private ArrayList<TableName> tableList;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual complete timestamp of the backup process
+  private long completeTs;
+
+  // total bytes for table backup image
+  private long totalBytes;
+
+  // total bytes for the backed-up logs for incremental backup
+  private long logBytes;
+
+  // the region server timestamp for tables:
+  // <table, <rs, timestamp>>
+  private Map<TableName, HashMap<String, Long>> incrTimeRanges;
+
+  // dependency of this backup, including all the dependent images to do PIT recovery
+  private Map<String, BackupImage> dependency;
+
+  // the indicator of the image compaction
+  private boolean isCompacted = false;
+  /**
+   * Construct manifest for a ongoing backup.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupContext backupCtx) {
+    this.backupId = backupCtx.getBackupId();
+    this.type = backupCtx.getType();
+    this.rootDir = backupCtx.getTargetRootDir();
+    if (this.type == BackupType.INCREMENTAL) {
+      this.logBackupDir = backupCtx.getHLogTargetDir();
+      this.logBytes = backupCtx.getTotalBytesCopied();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    this.loadTableList(backupCtx.getTableNames());
+  }
+
+  /**
+   * Construct a table level manifest for a backup of the named table.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupContext backupCtx, TableName table) {
+    this.backupId = backupCtx.getBackupId();
+    this.type = backupCtx.getType();
+    this.rootDir = backupCtx.getTargetRootDir();
+    this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir();
+    if (this.type == BackupType.INCREMENTAL) {
+      this.logBackupDir = backupCtx.getHLogTargetDir();
+      this.logBytes = backupCtx.getTotalBytesCopied();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    List<TableName> tables = new ArrayList<TableName>();
+    tables.add(table);
+    this.loadTableList(tables);
+  }
+
+  /**
+   * Construct manifest from a backup directory.
+   * @param conf configuration
+   * @param backupPath backup path
+   * @throws BackupException exception
+   */
+
+  public BackupManifest(Configuration conf, Path backupPath) throws BackupException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Loading manifest from: " + backupPath.toString());
+    }
+    // The input backupDir may not exactly be the backup table dir.
+    // It could be the backup log dir where there is also a manifest file stored.
+    // This variable's purpose is to keep the correct and original location so
+    // that we can store/persist it.
+    this.tableBackupDir = backupPath.toString();
+    this.config = conf;
+    try {
+
+      FileSystem fs = backupPath.getFileSystem(conf);
+      FileStatus[] subFiles = FSUtils.listStatus(fs, backupPath);
+      if (subFiles == null) {
+        String errorMsg = backupPath.toString() + " does not exist";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg);
+      }
+      for (FileStatus subFile : subFiles) {
+        if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) {
+
+          // load and set manifest field from file content
+          FSDataInputStream in = fs.open(subFile.getPath());
+          long len = subFile.getLen();
+          byte[] pbBytes = new byte[(int) len];
+          in.readFully(pbBytes);
+          BackupProtos.BackupManifest proto = null;
+          try{
+            proto = parseFrom(pbBytes);
+          } catch(Exception e){
+            throw new BackupException(e);
+          }
+          this.version = proto.getVersion();
+          this.backupId = proto.getBackupId();
+          this.type = BackupType.valueOf(proto.getType().name());
+          // Here the parameter backupDir is where the manifest file is.
+          // There should always be a manifest file under:
+          // backupRootDir/namespace/table/backupId/.backup.manifest
+          this.rootDir = backupPath.getParent().getParent().getParent().toString();
+
+          Path p = backupPath.getParent();
+          if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
+            this.rootDir = p.getParent().toString();
+          } else {
+            this.rootDir = p.getParent().getParent().toString();
+          }
+
+          loadTableList(proto);
+          this.startTs = proto.getStartTs();
+          this.completeTs = proto.getCompleteTs();
+          this.totalBytes = proto.getTotalBytes();
+          if (this.type == BackupType.INCREMENTAL) {
+            this.logBytes = proto.getLogBytes();
+            //TODO: convert will be implemented by future jira
+          }
+
+          loadIncrementalTimestampMap(proto);
+          loadDependency(proto);
+          this.isCompacted = proto.getCompacted();
+          //TODO: merge will be implemented by future jira
+          LOG.debug("Loaded manifest instance from manifest file: "
+              + FSUtils.getPath(subFile.getPath()));
+          return;
+        }
+      }
+      String errorMsg = "No manifest file found in: " + backupPath.toString();
+      LOG.error(errorMsg);
+      throw new IOException(errorMsg);
+
+    } catch (IOException e) {
+      LOG.error(e);
+      throw new BackupException(e.getMessage());
+    }
+  }
+
+  private void loadIncrementalTimestampMap(BackupProtos.BackupManifest proto) {
+    List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
+    if(list == null || list.size() == 0) return;
+    this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
+    for(BackupProtos.TableServerTimestamp tst: list){
+      TableName tn = ProtobufUtil.toTableName(tst.getTable());
+      HashMap<String, Long> map = this.incrTimeRanges.get(tn);
+      if(map == null){
+        map = new HashMap<String, Long>();
+        this.incrTimeRanges.put(tn, map);
+      }
+      List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
+      for(BackupProtos.ServerTimestamp stm: listSt) {
+        map.put(stm.getServer(), stm.getTimestamp());
+      }
+    }
+  }
+
+  private void loadDependency(BackupProtos.BackupManifest proto) {
+    dependency = new HashMap<String, BackupImage>();
+    List<BackupProtos.BackupImage> list = proto.getDependentBackupImageList();
+    for (BackupProtos.BackupImage im : list) {
+      dependency.put(im.getBackupId(), BackupImage.fromProto(im));
+    }
+  }
+
+  private void loadTableList(BackupProtos.BackupManifest proto) {
+    this.tableList = new ArrayList<TableName>();
+    List<HBaseProtos.TableName> list = proto.getTableListList();
+    for (HBaseProtos.TableName name: list) {
+      this.tableList.add(ProtobufUtil.toTableName(name));
+    }
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  /**
+   * Loads table list.
+   * @param tableList Table list
+   */
+  private void loadTableList(List<TableName> tableList) {
+
+    this.tableList = this.getTableList();
+    if (this.tableList.size() > 0) {
+      this.tableList.clear();
+    }
+    for (int i = 0; i < tableList.size(); i++) {
+      this.tableList.add(tableList.get(i));
+    }
+
+    LOG.debug(tableList.size() + " tables exist in table set.");
+  }
+
+  /**
+   * Get the table set of this image.
+   * @return The table set list
+   */
+  public ArrayList<TableName> getTableList() {
+    if (this.tableList == null) {
+      this.tableList = new ArrayList<TableName>();
+    }
+    return this.tableList;
+  }
+
+  /**
+   * Persist the manifest file.
+   * @throws IOException IOException when storing the manifest file.
+   */
+
+  public void store(Configuration conf) throws BackupException {
+    byte[] data = toByteArray();
+    // write the file, overwrite if already exist
+    Path manifestFilePath =
+        new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir))
+            ,MANIFEST_FILE_NAME);
+    try {
+      FSDataOutputStream out =
+          manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);
+      out.write(data);
+      out.close();
+    } catch (IOException e) {
+      LOG.error(e);
+      throw new BackupException(e.getMessage());
+    }
+
+    LOG.debug("Manifestfilestored_to " + this.tableBackupDir != null ? this.tableBackupDir
+        : this.logBackupDir + Path.SEPARATOR + MANIFEST_FILE_NAME);
+  }
+
+  /**
+   * Protobuf serialization
+   * @return The filter serialized using pb
+   */
+  public byte[] toByteArray() {
+    BackupProtos.BackupManifest.Builder builder = BackupProtos.BackupManifest.newBuilder();
+    builder.setVersion(this.version);
+    builder.setBackupId(this.backupId);
+    builder.setType(BackupProtos.BackupType.valueOf(this.type.name()));
+    setTableList(builder);
+    builder.setStartTs(this.startTs);
+    builder.setCompleteTs(this.completeTs);
+    builder.setTotalBytes(this.totalBytes);
+    if (this.type == BackupType.INCREMENTAL) {
+      builder.setLogBytes(this.logBytes);
+    }
+    setIncrementalTimestampMap(builder);
+    setDependencyMap(builder);
+    builder.setCompacted(this.isCompacted);
+    return builder.build().toByteArray();
+  }
+
+  private void setIncrementalTimestampMap(BackupProtos.BackupManifest.Builder builder) {
+    if (this.incrTimeRanges == null) return;
+    for (Entry<TableName, HashMap<String,Long>> entry: this.incrTimeRanges.entrySet()) {
+      TableName key = entry.getKey();
+      HashMap<String, Long> value = entry.getValue();
+      BackupProtos.TableServerTimestamp.Builder tstBuilder =
+          BackupProtos.TableServerTimestamp.newBuilder();
+      tstBuilder.setTable(ProtobufUtil.toProtoTableName(key));
+
+      for (String s : value.keySet()) {
+        BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
+        stBuilder.setServer(s);
+        stBuilder.setTimestamp(value.get(s));
+        tstBuilder.addServerTimestamp(stBuilder.build());
+      }
+      builder.addTstMap(tstBuilder.build());
+    }
+  }
+
+  private void setDependencyMap(BackupProtos.BackupManifest.Builder builder) {
+    for (BackupImage image: getDependency().values()) {
+      builder.addDependentBackupImage(image.toProto());
+    }
+  }
+
+  private void setTableList(BackupProtos.BackupManifest.Builder builder) {
+    for(TableName name: tableList){
+      builder.addTableList(ProtobufUtil.toProtoTableName(name));
+    }
+  }
+
+  /**
+   * Parse protobuf from byte array
+   * @param pbBytes A pb serialized BackupManifest instance
+   * @return An instance of  made from <code>bytes</code>
+   * @throws DeserializationException
+   */
+  private static BackupProtos.BackupManifest parseFrom(final byte[] pbBytes)
+      throws DeserializationException {
+    BackupProtos.BackupManifest proto;
+    try {
+      proto = BackupProtos.BackupManifest.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return proto;
+  }
+
+  /**
+   * Get manifest file version
+   * @return version
+   */
+  public String getVersion() {
+    return version;
+  }
+
+  /**
+   * Get this backup image.
+   * @return the backup image.
+   */
+  public BackupImage getBackupImage() {
+    return this.getDependency().get(this.backupId);
+  }
+
+  /**
+   * Add dependent backup image for this backup.
+   * @param image The direct dependent backup image
+   */
+  public void addDependentImage(BackupImage image) {
+    this.getDependency().get(this.backupId).addAncestor(image);
+    this.setDependencyMap(this.getDependency(), image);
+  }
+
+
+
+  /**
+   * Get all dependent backup images. The image of this backup is also contained.
+   * @return The dependent backup images map
+   */
+  public Map<String, BackupImage> getDependency() {
+    if (this.dependency == null) {
+      this.dependency = new HashMap<String, BackupImage>();
+      LOG.debug(this.rootDir + " " + this.backupId + " " + this.type);
+      this.dependency.put(this.backupId,
+        new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
+            this.completeTs));
+    }
+    return this.dependency;
+  }
+
+  /**
+   * Set the incremental timestamp map directly.
+   * @param incrTimestampMap timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
+    this.incrTimeRanges = incrTimestampMap;
+  }
+
+
+  public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    if (this.incrTimeRanges == null) {
+      this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
+    }
+    return this.incrTimeRanges;
+  }
+
+
+  /**
+   * Get the image list of this backup for restore in time order.
+   * @param reverse If true, then output in reverse order, otherwise in time order from old to new
+   * @return the backup image list for restore in time order
+   */
+  public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
+    TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
+    for (BackupImage image : this.getDependency().values()) {
+      restoreImages.put(Long.valueOf(image.startTs), image);
+    }
+    return new ArrayList<BackupImage>(reverse ? (restoreImages.descendingMap().values())
+        : (restoreImages.values()));
+  }
+
+  /**
+   * Get the dependent image list for a specific table of this backup in time order from old to new
+   * if want to restore to this backup image level.
+   * @param table table
+   * @return the backup image list for a table in time order
+   */
+  public ArrayList<BackupImage> getDependentListByTable(TableName table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> imageList = getRestoreDependentList(true);
+    for (BackupImage image : imageList) {
+      if (image.hasTable(table)) {
+        tableImageList.add(image);
+        if (image.getType() == BackupType.FULL) {
+          break;
+        }
+      }
+    }
+    Collections.reverse(tableImageList);
+    return tableImageList;
+  }
+
+  /**
+   * Get the full dependent image list in the whole dependency scope for a specific table of this
+   * backup in time order from old to new.
+   * @param table table
+   * @return the full backup image list for a table in time order in the whole scope of the
+   *         dependency of this image
+   */
+  public ArrayList<BackupImage> getAllDependentListByTable(TableName table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> imageList = getRestoreDependentList(false);
+    for (BackupImage image : imageList) {
+      if (image.hasTable(table)) {
+        tableImageList.add(image);
+      }
+    }
+    return tableImageList;
+  }
+
+
+  /**
+   * Recursively set the dependency map of the backup images.
+   * @param map The dependency map
+   * @param image The backup image
+   */
+  private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
+    if (image == null) {
+      return;
+    } else {
+      map.put(image.getBackupId(), image);
+      for (BackupImage img : image.getAncestors()) {
+        setDependencyMap(map, img);
+      }
+    }
+  }
+
+  /**
+   * Check whether backup image1 could cover backup image2 or not.
+   * @param image1 backup image 1
+   * @param image2 backup image 2
+   * @return true if image1 can cover image2, otherwise false
+   */
+  public static boolean canCoverImage(BackupImage image1, BackupImage image2) {
+    // image1 can cover image2 only when the following conditions are satisfied:
+    // - image1 must not be an incremental image;
+    // - image1 must be taken after image2 has been taken;
+    // - table set of image1 must cover the table set of image2.
+    if (image1.getType() == BackupType.INCREMENTAL) {
+      return false;
+    }
+    if (image1.getStartTs() < image2.getStartTs()) {
+      return false;
+    }
+    List<TableName> image1TableList = image1.getTableNames();
+    List<TableName> image2TableList = image2.getTableNames();
+    boolean found = false;
+    for (int i = 0; i < image2TableList.size(); i++) {
+      found = false;
+      for (int j = 0; j < image1TableList.size(); j++) {
+        if (image2TableList.get(i).equals(image1TableList.get(j))) {
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        return false;
+      }
+    }
+
+    LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId());
+    return true;
+  }
+
+  /**
+   * Check whether backup image set could cover a backup image or not.
+   * @param fullImages The backup image set
+   * @param image The target backup image
+   * @return true if fullImages can cover image, otherwise false
+   */
+  public static boolean canCoverImage(ArrayList<BackupImage> fullImages, BackupImage image) {
+    // fullImages can cover image only when the following conditions are satisfied:
+    // - each image of fullImages must not be an incremental image;
+    // - each image of fullImages must be taken after image has been taken;
+    // - sum table set of fullImages must cover the table set of image.
+    for (BackupImage image1 : fullImages) {
+      if (image1.getType() == BackupType.INCREMENTAL) {
+        return false;
+      }
+      if (image1.getStartTs() < image.getStartTs()) {
+        return false;
+      }
+    }
+
+    ArrayList<String> image1TableList = new ArrayList<String>();
+    for (BackupImage image1 : fullImages) {
+      List<TableName> tableList = image1.getTableNames();
+      for (TableName table : tableList) {
+        image1TableList.add(table.getNameAsString());
+      }
+    }
+    ArrayList<String> image2TableList = new ArrayList<String>();
+    List<TableName> tableList = image.getTableNames();
+    for (TableName table : tableList) {
+      image2TableList.add(table.getNameAsString());
+    }
+
+    for (int i = 0; i < image2TableList.size(); i++) {
+      if (image1TableList.contains(image2TableList.get(i)) == false) {
+        return false;
+      }
+    }
+
+    LOG.debug("Full image set can cover image " + image.getBackupId());
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
new file mode 100644
index 0000000..d0ce059
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
@@ -0,0 +1,46 @@
+/**
+ * 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.backup.impl;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * HConstants holds a bunch of HBase Backup and Restore constants
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public final class BackupRestoreConstants {
+
+
+  // delimiter in tablename list in restore command
+  public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
+
+  public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
+
+  public static final String BACKUPID_PREFIX = "backup_";
+
+  public static enum BackupCommand {
+    CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP,
+  }
+
+  private BackupRestoreConstants() {
+    // Can't be instantiated with this ctor.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
new file mode 100644
index 0000000..8b8a83f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSnapshotCopy.java
@@ -0,0 +1,42 @@
+/**
+ * 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.backup.impl;
+
+import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
+
+/* this class will be extended in future jira to support progress report */
+public class BackupSnapshotCopy extends ExportSnapshot {
+  private BackupHandler backupHandler;
+  private String table;
+
+  public BackupSnapshotCopy(BackupHandler backupHandler, String table) {
+    super();
+    this.backupHandler = backupHandler;
+    this.table = table;
+  }
+
+  public BackupHandler getBackupHandler() {
+    return this.backupHandler;
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab491d4a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java
new file mode 100644
index 0000000..6e54994
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupStatus.java
@@ -0,0 +1,105 @@
+/**
+ * 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.backup.impl;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+
+/**
+ * Backup status and related information encapsulated for a table.
+ * At this moment only TargetDir and SnapshotName is encapsulated here.
+ * future Jira will be implemented for progress, bytesCopies, phase, etc.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupStatus implements Serializable {
+
+  private static final long serialVersionUID = -5968397963548535982L;
+
+  // table name for backup
+  private TableName table;
+
+  // target directory of the backup image for this table
+  private String targetDir;
+
+  // snapshot name for offline/online snapshot
+  private String snapshotName = null;
+
+  public BackupStatus() {
+
+  }
+
+  public BackupStatus(TableName table, String targetRootDir, String backupId) {
+    this.table = table;
+    this.targetDir = HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+  }
+
+  public String getSnapshotName() {
+    return snapshotName;
+  }
+
+  public void setSnapshotName(String snapshotName) {
+    this.snapshotName = snapshotName;
+  }
+
+  public String getTargetDir() {
+    return targetDir;
+  }
+
+  public TableName getTable() {
+    return table;
+  }
+
+  public void setTable(TableName table) {
+    this.table = table;
+  }
+
+  public void setTargetDir(String targetDir) {
+    this.targetDir = targetDir;
+  }
+
+  public static BackupStatus convert(BackupProtos.TableBackupStatus proto)
+  {
+    BackupStatus bs = new BackupStatus();
+    bs.setTable(ProtobufUtil.toTableName(proto.getTable()));
+    bs.setTargetDir(proto.getTargetDir());
+    if(proto.hasSnapshot()){
+      bs.setSnapshotName(proto.getSnapshot());
+    }
+    return bs;
+  }
+
+  public BackupProtos.TableBackupStatus toProto() {
+    BackupProtos.TableBackupStatus.Builder builder =
+        BackupProtos.TableBackupStatus.newBuilder();
+    if(snapshotName != null) {
+      builder.setSnapshot(snapshotName);
+    }
+    builder.setTable(ProtobufUtil.toProtoTableName(table));
+    builder.setTargetDir(targetDir);
+    return builder.build();
+  }
+}


[36/50] [abbrv] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/99955a32/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>

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

Posted by en...@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/99955a32
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/99955a32
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/99955a32

Branch: refs/heads/HBASE-7912
Commit: 99955a3240c5032daae471cacebe595134f71fc3
Parents: 793babf
Author: chenheng <ch...@apache.org>
Authored: Sat Feb 27 08:36:59 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Sat Feb 27 08:36:59 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 |   37 +
 .../hbase/zookeeper/SplitOrMergeTracker.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, 4824 insertions(+), 1200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/99955a32/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/99955a32/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/99955a32/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/99955a32/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/99955a32/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;
+  }
 }


[39/50] [abbrv] hbase git commit: HBASE-15181 adds TestCompactionPolicy which was missing in first commit

Posted by en...@apache.org.
HBASE-15181 adds TestCompactionPolicy which was missing in first commit


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

Branch: refs/heads/HBASE-7912
Commit: 03ffb30efe341c226a19b4e80ec0e3352e55806c
Parents: f7f96b9
Author: tedyu <yu...@gmail.com>
Authored: Fri Feb 26 19:58:33 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Feb 26 19:58:33 2016 -0800

----------------------------------------------------------------------
 .../regionserver/TestCompactionPolicy.java      | 207 +++++++++++++++++++
 1 file changed, 207 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/03ffb30e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
new file mode 100644
index 0000000..f5f0926
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -0,0 +1,207 @@
+/**
+ * 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 com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+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.regionserver.compactions.CompactionConfiguration;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestCompactionPolicy {
+  private final static Log LOG = LogFactory.getLog(TestDefaultCompactSelection.class);
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  protected Configuration conf;
+  protected HStore store;
+  private static final String DIR = TEST_UTIL.getDataTestDir(
+    TestDefaultCompactSelection.class.getSimpleName()).toString();
+  protected static Path TEST_FILE;
+  protected static final int minFiles = 3;
+  protected static final int maxFiles = 5;
+
+  protected static final long minSize = 10;
+  protected static final long maxSize = 2100;
+
+  private FSHLog hlog;
+  private HRegion region;
+
+  @Before
+  public void setUp() throws Exception {
+    config();
+    initialize();
+  }
+
+  /**
+   * setup config values necessary for store
+   */
+  protected void config() {
+    this.conf = TEST_UTIL.getConfiguration();
+    this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
+    this.conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, minFiles);
+    this.conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, maxFiles);
+    this.conf.setLong(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_SIZE_KEY, minSize);
+    this.conf.setLong(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_SIZE_KEY, maxSize);
+    this.conf.setFloat(CompactionConfiguration.HBASE_HSTORE_COMPACTION_RATIO_KEY, 1.0F);
+  }
+
+  /**
+   * Setting up a Store
+   * @throws IOException with error
+   */
+  protected void initialize() throws IOException {
+    Path basedir = new Path(DIR);
+    String logName = "logs";
+    Path logdir = new Path(DIR, logName);
+    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family"));
+    FileSystem fs = FileSystem.get(conf);
+
+    fs.delete(logdir, true);
+
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("table")));
+    htd.addFamily(hcd);
+    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+
+    hlog = new FSHLog(fs, basedir, logName, conf);
+    region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
+    region.close();
+    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
+    region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
+
+    store = new HStore(region, hcd, conf);
+
+    TEST_FILE = region.getRegionFileSystem().createTempName();
+    fs.createNewFile(TEST_FILE);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    IOException ex = null;
+    try {
+      region.close();
+    } catch (IOException e) {
+      LOG.warn("Caught Exception", e);
+      ex = e;
+    }
+    try {
+      hlog.close();
+    } catch (IOException e) {
+      LOG.warn("Caught Exception", e);
+      ex = e;
+    }
+    if (ex != null) {
+      throw ex;
+    }
+  }
+
+  ArrayList<Long> toArrayList(long... numbers) {
+    ArrayList<Long> result = new ArrayList<Long>();
+    for (long i : numbers) {
+      result.add(i);
+    }
+    return result;
+  }
+
+  List<StoreFile> sfCreate(long... sizes) throws IOException {
+    ArrayList<Long> ageInDisk = new ArrayList<Long>();
+    for (int i = 0; i < sizes.length; i++) {
+      ageInDisk.add(0L);
+    }
+    return sfCreate(toArrayList(sizes), ageInDisk);
+  }
+
+  List<StoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk) throws IOException {
+    return sfCreate(false, sizes, ageInDisk);
+  }
+
+  List<StoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
+    ArrayList<Long> ageInDisk = new ArrayList<Long>(sizes.length);
+    for (int i = 0; i < sizes.length; i++) {
+      ageInDisk.add(0L);
+    }
+    return sfCreate(isReference, toArrayList(sizes), ageInDisk);
+  }
+
+  List<StoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
+      throws IOException {
+    List<StoreFile> ret = Lists.newArrayList();
+    for (int i = 0; i < sizes.size(); i++) {
+      ret.add(new MockStoreFile(TEST_UTIL, TEST_FILE, sizes.get(i), ageInDisk.get(i), isReference,
+          i));
+    }
+    return ret;
+  }
+
+  long[] getSizes(List<StoreFile> sfList) {
+    long[] aNums = new long[sfList.size()];
+    for (int i = 0; i < sfList.size(); ++i) {
+      aNums[i] = sfList.get(i).getReader().length();
+    }
+    return aNums;
+  }
+
+  void compactEquals(List<StoreFile> candidates, long... expected) throws IOException {
+    compactEquals(candidates, false, false, expected);
+  }
+
+  void compactEquals(List<StoreFile> candidates, boolean forcemajor, long... expected)
+      throws IOException {
+    compactEquals(candidates, forcemajor, false, expected);
+  }
+
+  void compactEquals(List<StoreFile> candidates, boolean forcemajor, boolean isOffPeak,
+      long... expected) throws IOException {
+    store.forceMajor = forcemajor;
+    // Test Default compactions
+    CompactionRequest result =
+        ((RatioBasedCompactionPolicy) store.storeEngine.getCompactionPolicy()).selectCompaction(
+          candidates, new ArrayList<StoreFile>(), false, isOffPeak, forcemajor);
+    List<StoreFile> actual = new ArrayList<StoreFile>(result.getFiles());
+    if (isOffPeak && !forcemajor) {
+      Assert.assertTrue(result.isOffPeak());
+    }
+    Assert.assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
+    store.forceMajor = false;
+  }
+}


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

Posted by en...@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-7912
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,


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

Posted by en...@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-7912
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>


[17/50] [abbrv] hbase git commit: HBASE-15311 Prevent NPE in BlockCacheViewTmpl.

Posted by en...@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-7912
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;
     }


[21/50] [abbrv] hbase git commit: HBASE-15128 Disable region splits and merges switch in master

Posted by en...@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-7912
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;
+  }
 }


[05/50] [abbrv] hbase git commit: HBASE-15302 Reenable the other tests disabled by HBASE-14678

Posted by en...@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